From f8b6bb7f1ab8720ca4f2d766831d8f243dd27085 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Nov 2016 15:41:13 -0800 Subject: [PATCH 001/279] Add JUnit category for stateful ParDo tests --- .../beam/sdk/testing/UsesStatefulParDo.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.java new file mode 100644 index 000000000000..8bd633036992 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.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; + +import org.apache.beam.sdk.transforms.ParDo; + +/** + * Category tag for validation tests which utilize stateful {@link ParDo}. + */ +public interface UsesStatefulParDo {} From e85cea78253d2f316a18d95d65aabc1176448841 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 15 Nov 2016 21:33:01 -0800 Subject: [PATCH 002/279] Reject stateful DoFn in ApexRunner --- runners/apex/pom.xml | 1 + .../ParDoBoundMultiTranslator.java | 67 ++++++++++++------- .../translation/ParDoBoundTranslator.java | 46 ++++++++----- 3 files changed, 74 insertions(+), 40 deletions(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 5478b24f046c..d0b0fdf1d208 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -185,6 +185,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.UsesStatefulParDo none true diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java index 7c91b91d36aa..fed5f4b37480 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java @@ -23,17 +23,17 @@ import com.datatorrent.api.Operator; import com.datatorrent.api.Operator.OutputPort; import com.google.common.collect.Maps; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - +import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.PCollection; @@ -53,20 +53,35 @@ class ParDoBoundMultiTranslator @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { - OldDoFn doFn = transform.getFn(); + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + ApexRunner.class.getSimpleName())); + } + OldDoFn oldDoFn = transform.getFn(); PCollectionTuple output = context.getOutput(); PCollection input = context.getInput(); List> sideInputs = transform.getSideInputs(); Coder inputCoder = input.getCoder(); - WindowedValueCoder wvInputCoder = FullWindowedValueCoder.of(inputCoder, - input.getWindowingStrategy().getWindowFn().windowCoder()); + WindowedValueCoder wvInputCoder = + FullWindowedValueCoder.of( + inputCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); - ApexParDoOperator operator = new ApexParDoOperator<>( - context.getPipelineOptions(), - doFn, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), - context.>getInput().getWindowingStrategy(), sideInputs, wvInputCoder, - context.stateInternalsFactory() - ); + ApexParDoOperator operator = + new ApexParDoOperator<>( + context.getPipelineOptions(), + oldDoFn, + transform.getMainOutputTag(), + transform.getSideOutputTags().getAll(), + context.>getInput().getWindowingStrategy(), + sideInputs, + wvInputCoder, + context.stateInternalsFactory()); Map, PCollection> outputs = output.getAll(); Map, OutputPort> ports = Maps.newHashMapWithExpectedSize(outputs.size()); @@ -91,7 +106,9 @@ public void translate(ParDo.BoundMulti transform, TranslationCo } } - static void addSideInputs(ApexParDoOperator operator, List> sideInputs, + static void addSideInputs( + ApexParDoOperator operator, + List> sideInputs, TranslationContext context) { Operator.InputPort[] sideInputPorts = {operator.sideInput1}; if (sideInputs.size() > sideInputPorts.length) { @@ -105,8 +122,8 @@ static void addSideInputs(ApexParDoOperator operator, List unionSideInputs(List> sideInputs, - TranslationContext context) { + private static PCollection unionSideInputs( + List> sideInputs, TranslationContext context) { checkArgument(sideInputs.size() > 1, "requires multiple side inputs"); // flatten and assign union tag List> sourceCollections = new ArrayList<>(); @@ -115,13 +132,16 @@ private static PCollection unionSideInputs(List> sideInput for (int i = 0; i < sideInputs.size(); i++) { PCollectionView sideInput = sideInputs.get(i); PCollection sideInputCollection = context.getViewInput(sideInput); - if (!sideInputCollection.getWindowingStrategy().equals( - firstSideInput.getWindowingStrategy())) { + if (!sideInputCollection + .getWindowingStrategy() + .equals(firstSideInput.getWindowingStrategy())) { // TODO: check how to handle this in stream codec //String msg = "Multiple side inputs with different window strategies."; //throw new UnsupportedOperationException(msg); - LOG.warn("Side inputs union with different windowing strategies {} {}", - firstSideInput.getWindowingStrategy(), sideInputCollection.getWindowingStrategy()); + LOG.warn( + "Side inputs union with different windowing strategies {} {}", + firstSideInput.getWindowingStrategy(), + sideInputCollection.getWindowingStrategy()); } if (!sideInputCollection.getCoder().equals(firstSideInput.getCoder())) { String msg = "Multiple side inputs with different coders."; @@ -131,12 +151,11 @@ private static PCollection unionSideInputs(List> sideInput unionTags.put(sideInputCollection, i); } - PCollection resultCollection = FlattenPCollectionTranslator.intermediateCollection( - firstSideInput, firstSideInput.getCoder()); - FlattenPCollectionTranslator.flattenCollections(sourceCollections, unionTags, resultCollection, - context); + PCollection resultCollection = + FlattenPCollectionTranslator.intermediateCollection( + firstSideInput, firstSideInput.getCoder()); + FlattenPCollectionTranslator.flattenCollections( + sourceCollections, unionTags, resultCollection, context); return resultCollection; - } - } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java index c1ebbd5df739..7a918a71e431 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java @@ -19,12 +19,13 @@ package org.apache.beam.runners.apex.translation; import java.util.List; - +import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.PCollection; @@ -32,33 +33,46 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -/** - * {@link ParDo.Bound} is translated to {link ApexParDoOperator} that wraps the {@link DoFn}. - */ -class ParDoBoundTranslator implements - TransformTranslator> { +/** {@link ParDo.Bound} is translated to {link ApexParDoOperator} that wraps the {@link DoFn}. */ +class ParDoBoundTranslator + implements TransformTranslator> { private static final long serialVersionUID = 1L; @Override public void translate(ParDo.Bound transform, TranslationContext context) { - OldDoFn doFn = transform.getFn(); + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + ApexRunner.class.getSimpleName())); + } + OldDoFn oldDoFn = transform.getOldFn(); PCollection output = context.getOutput(); PCollection input = context.getInput(); List> sideInputs = transform.getSideInputs(); Coder inputCoder = input.getCoder(); - WindowedValueCoder wvInputCoder = FullWindowedValueCoder.of(inputCoder, - input.getWindowingStrategy().getWindowFn().windowCoder()); + WindowedValueCoder wvInputCoder = + FullWindowedValueCoder.of( + inputCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); - ApexParDoOperator operator = new ApexParDoOperator<>( - context.getPipelineOptions(), - doFn, new TupleTag(), TupleTagList.empty().getAll() /*sideOutputTags*/, - output.getWindowingStrategy(), sideInputs, wvInputCoder, - context.stateInternalsFactory() - ); + ApexParDoOperator operator = + new ApexParDoOperator<>( + context.getPipelineOptions(), + oldDoFn, + new TupleTag(), + TupleTagList.empty().getAll() /*sideOutputTags*/, + output.getWindowingStrategy(), + sideInputs, + wvInputCoder, + context.stateInternalsFactory()); context.addOperator(operator, operator.output); context.addStream(context.getInput(), operator.input); if (!sideInputs.isEmpty()) { - ParDoBoundMultiTranslator.addSideInputs(operator, sideInputs, context); + ParDoBoundMultiTranslator.addSideInputs(operator, sideInputs, context); } } } From 8d715689dd5283b7b180c0b9ec4e188abba140f5 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Nov 2016 15:41:13 -0800 Subject: [PATCH 003/279] Add JUnit category for stateful ParDo tests --- .../beam/sdk/testing/UsesStatefulParDo.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.java new file mode 100644 index 000000000000..8bd633036992 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesStatefulParDo.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; + +import org.apache.beam.sdk.transforms.ParDo; + +/** + * Category tag for validation tests which utilize stateful {@link ParDo}. + */ +public interface UsesStatefulParDo {} From b0d07d74f7805ee1d30fdedf54c089790d63d898 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 15 Nov 2016 21:33:13 -0800 Subject: [PATCH 004/279] Reject stateful DoFn in SparkRunner --- runners/spark/pom.xml | 1 + .../translation/TransformTranslator.java | 23 +++++++++++++++++++ 2 files changed, 24 insertions(+) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 4c5b3f5df2b3..88223e2b49c0 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -72,6 +72,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.UsesStatefulParDo 1 false true diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index c902ee30a9a9..60d668e3b1bd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -31,6 +31,7 @@ import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.beam.runners.core.AssignWindowsDoFn; +import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.runners.spark.io.SourceRDD; @@ -47,12 +48,14 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; 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.OldDoFn; 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.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -225,6 +228,16 @@ private static TransformEvaluator return new TransformEvaluator>() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + SparkRunner.class.getSimpleName())); + } @SuppressWarnings("unchecked") JavaRDD> inRDD = ((BoundedDataset) context.borrowDataset(transform)).getRDD(); @@ -247,6 +260,16 @@ public void evaluate(ParDo.Bound transform, EvaluationContext c return new TransformEvaluator>() { @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + SparkRunner.class.getSimpleName())); + } @SuppressWarnings("unchecked") JavaRDD> inRDD = ((BoundedDataset) context.borrowDataset(transform)).getRDD(); From c80554b83426a585c762143e0ad533a73c2c3f0f Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Mon, 21 Nov 2016 16:33:07 -0800 Subject: [PATCH 005/279] Update StarterPipeline Convert StarterPipeline ParDo to MapElements. Use the new DoFn for non-outputting transforms. --- .../src/main/java/StarterPipeline.java | 18 ++++++++++-------- .../src/main/java/it/pkg/StarterPipeline.java | 18 ++++++++++-------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java index 0b21aa650a8d..d6afdecf11db 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/src/main/java/StarterPipeline.java @@ -20,13 +20,15 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * A starter example for writing Google Cloud Dataflow programs. + * A starter example for writing Beam programs. * *

The example takes two strings, converts them to their upper-case * representation and logs them. @@ -39,7 +41,7 @@ * Platform, you should specify the following command-line options: * --project= * --stagingLocation= - * --runner=BlockingDataflowRunner + * --runner=DataflowRunner */ public class StarterPipeline { private static final Logger LOG = LoggerFactory.getLogger(StarterPipeline.class); @@ -49,14 +51,14 @@ public static void main(String[] args) { PipelineOptionsFactory.fromArgs(args).withValidation().create()); p.apply(Create.of("Hello", "World")) - .apply(ParDo.of(new OldDoFn() { + .apply(MapElements.via(new SimpleFunction() { @Override - public void processElement(ProcessContext c) { - c.output(c.element().toUpperCase()); + public String apply(String input) { + return input.toUpperCase(); } })) - .apply(ParDo.of(new OldDoFn() { - @Override + .apply(ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { LOG.info(c.element()); } diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java index b332442cadcd..4ae92e8ce6d1 100644 --- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java +++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/src/main/java/it/pkg/StarterPipeline.java @@ -20,13 +20,15 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * A starter example for writing Google Cloud Dataflow programs. + * A starter example for writing Beam programs. * *

The example takes two strings, converts them to their upper-case * representation and logs them. @@ -39,7 +41,7 @@ * Platform, you should specify the following command-line options: * --project= * --stagingLocation= - * --runner=BlockingDataflowRunner + * --runner=DataflowRunner */ public class StarterPipeline { private static final Logger LOG = LoggerFactory.getLogger(StarterPipeline.class); @@ -49,14 +51,14 @@ public static void main(String[] args) { PipelineOptionsFactory.fromArgs(args).withValidation().create()); p.apply(Create.of("Hello", "World")) - .apply(ParDo.of(new OldDoFn() { + .apply(MapElements.via(new SimpleFunction() { @Override - public void processElement(ProcessContext c) { - c.output(c.element().toUpperCase()); + public String apply(String input) { + return input.toUpperCase(); } })) - .apply(ParDo.of(new OldDoFn() { - @Override + .apply(ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { LOG.info(c.element()); } From 9c2d5da7c659a2603d37c492ff44f4a9cda387fe Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 15 Nov 2016 21:33:28 -0800 Subject: [PATCH 006/279] Reject stateful DoFn in FlinkRunner --- runners/flink/runner/pom.xml | 1 + .../FlinkBatchTransformTranslators.java | 34 ++++++++++++++++--- .../FlinkStreamingTransformTranslators.java | 25 +++++++++++++- 3 files changed, 55 insertions(+), 5 deletions(-) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index adcb3debb52c..c060c254dbf7 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -53,6 +53,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.UsesStatefulParDo none true 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 935a9ac439b9..474d4e397cfb 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 @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMergingNonShuffleReduceFunction; @@ -46,6 +47,7 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineFnBase; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.OldDoFn; @@ -54,6 +56,7 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.join.UnionCoder; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -487,11 +490,23 @@ private static class ParDoBoundTranslatorBatch @Override public void translateNode( ParDo.Bound transform, + FlinkBatchTranslationContext context) { + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - final OldDoFn doFn = transform.getFn(); + final OldDoFn oldDoFn = transform.getFn(); TypeInformation> typeInformation = context.getTypeInfo(context.getOutput(transform)); @@ -507,7 +522,7 @@ public void translateNode( FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>( - doFn, + oldDoFn, context.getOutput(transform).getWindowingStrategy(), sideInputStrategies, context.getPipelineOptions()); @@ -533,10 +548,21 @@ private static class ParDoBoundMultiTranslatorBatch public void translateNode( ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - final OldDoFn doFn = transform.getFn(); + final OldDoFn oldDoFn = transform.getFn(); Map, PCollection> outputs = context.getOutput(transform).getAll(); @@ -584,7 +610,7 @@ public void translateNode( @SuppressWarnings("unchecked") FlinkMultiOutputDoFnFunction doFnWrapper = new FlinkMultiOutputDoFnFunction( - doFn, + oldDoFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(), 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 687e9c8b3e44..40dfbb990b50 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 @@ -21,7 +21,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; - import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -51,6 +50,7 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.OldDoFn; @@ -58,6 +58,7 @@ 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.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -311,6 +312,17 @@ public void translateNode( ParDo.Bound transform, FlinkStreamingTranslationContext context) { + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + WindowingStrategy windowingStrategy = context.getOutput(transform).getWindowingStrategy(); @@ -460,6 +472,17 @@ public void translateNode( ParDo.BoundMulti transform, FlinkStreamingTranslationContext context) { + DoFn doFn = transform.getNewFn(); + if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + // we assume that the transformation does not change the windowing strategy. WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); From e6870a6dc10e4ad52a911c316137a9f7731a9194 Mon Sep 17 00:00:00 2001 From: bchambers Date: Tue, 22 Nov 2016 11:37:23 -0800 Subject: [PATCH 007/279] Simplify the API for managing MetricsEnvironment 1. setCurrentContainer returns the previous MetricsEnvironment 2. setCurrentContainer(null) resets the thread local 3. scopedCurrentContainer sets the container and returns a Closeable to reset the previous container. --- .../runners/direct/TransformExecutor.java | 5 +- .../beam/sdk/metrics/MetricsEnvironment.java | 60 +++++++++++++++---- .../sdk/metrics/MetricsEnvironmentTest.java | 8 +-- .../apache/beam/sdk/metrics/MetricsTest.java | 6 +- 4 files changed, 56 insertions(+), 23 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java index 1704955b538a..fb31cc97f74d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import java.io.Closeable; import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; @@ -89,8 +90,7 @@ private TransformExecutor( @Override public void run() { MetricsContainer metricsContainer = new MetricsContainer(transform.getFullName()); - MetricsEnvironment.setMetricsContainer(metricsContainer); - try { + try (Closeable metricsScope = MetricsEnvironment.scopedMetricsContainer(metricsContainer)) { Collection> enforcements = new ArrayList<>(); for (ModelEnforcementFactory enforcementFactory : modelEnforcements) { ModelEnforcement enforcement = enforcementFactory.forBundle(inputBundle, transform); @@ -117,7 +117,6 @@ public void run() { // Report the physical metrics from the end of this step. context.getMetrics().commitPhysical(inputBundle, metricsContainer.getCumulative()); - MetricsEnvironment.unsetMetricsContainer(); transformEvaluationState.complete(this); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java index ef2660a83994..7c06cbf7150d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.metrics; +import java.io.Closeable; +import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; import org.slf4j.Logger; @@ -29,11 +31,13 @@ * returned objects to create and modify metrics. * *

The runner should create {@link MetricsContainer} for each context in which metrics are - * reported (by step and name) and call {@link #setMetricsContainer} before invoking any code that - * may update metrics within that step. + * reported (by step and name) and call {@link #setCurrentContainer} before invoking any code that + * may update metrics within that step. It should call {@link #setCurrentContainer} again to restore + * the previous container. * - *

The runner should call {@link #unsetMetricsContainer} (or {@link #setMetricsContainer} back to - * the previous value) when exiting code that set the metrics container. + *

Alternatively, the runner can use {@link #scopedMetricsContainer(MetricsContainer)} to set the + * container for the current thread and get a {@link Closeable} that will restore the previous + * container when closed. */ public class MetricsEnvironment { @@ -45,15 +49,20 @@ public class MetricsEnvironment { private static final ThreadLocal CONTAINER_FOR_THREAD = new ThreadLocal(); - /** Set the {@link MetricsContainer} for the current thread. */ - public static void setMetricsContainer(MetricsContainer container) { - CONTAINER_FOR_THREAD.set(container); - } - - - /** Clear the {@link MetricsContainer} for the current thread. */ - public static void unsetMetricsContainer() { - CONTAINER_FOR_THREAD.remove(); + /** + * Set the {@link MetricsContainer} for the current thread. + * + * @return The previous container for the current thread. + */ + @Nullable + public static MetricsContainer setCurrentContainer(@Nullable MetricsContainer container) { + MetricsContainer previous = getCurrentContainer(); + if (container == null) { + CONTAINER_FOR_THREAD.remove(); + } else { + CONTAINER_FOR_THREAD.set(container); + } + return previous; } /** Called by the run to indicate whether metrics reporting is supported. */ @@ -61,6 +70,31 @@ public static void setMetricsSupported(boolean supported) { METRICS_SUPPORTED.set(supported); } + /** + * Set the {@link MetricsContainer} for the current thread. + * + * @return A {@link Closeable} that will reset the current container to the previous + * {@link MetricsContainer} when closed. + */ + public static Closeable scopedMetricsContainer(MetricsContainer container) { + return new ScopedContainer(container); + } + + private static class ScopedContainer implements Closeable { + + @Nullable + private final MetricsContainer oldContainer; + + private ScopedContainer(MetricsContainer newContainer) { + this.oldContainer = setCurrentContainer(newContainer); + } + + @Override + public void close() throws IOException { + setCurrentContainer(oldContainer); + } + } + /** * Return the {@link MetricsContainer} for the current thread. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java index 4200a200fd38..0ce17b4bb9c3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java @@ -35,7 +35,7 @@ public class MetricsEnvironmentTest { @After public void teardown() { - MetricsEnvironment.unsetMetricsContainer(); + MetricsEnvironment.setCurrentContainer(null); } @Test @@ -44,11 +44,11 @@ public void testUsesAppropriateMetricsContainer() { MetricsContainer c1 = new MetricsContainer("step1"); MetricsContainer c2 = new MetricsContainer("step2"); - MetricsEnvironment.setMetricsContainer(c1); + MetricsEnvironment.setCurrentContainer(c1); counter.inc(); - MetricsEnvironment.setMetricsContainer(c2); + MetricsEnvironment.setCurrentContainer(c2); counter.dec(); - MetricsEnvironment.unsetMetricsContainer(); + MetricsEnvironment.setCurrentContainer(null); MetricUpdates updates1 = c1.getUpdates(); MetricUpdates updates2 = c2.getUpdates(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java index d11b44ddb19f..732cb346d20a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java @@ -37,7 +37,7 @@ public class MetricsTest { @After public void tearDown() { - MetricsEnvironment.unsetMetricsContainer(); + MetricsEnvironment.setCurrentContainer(null); } @Test @@ -61,7 +61,7 @@ public void counterWithoutContainer() { @Test public void distributionToCell() { MetricsContainer container = new MetricsContainer("step"); - MetricsEnvironment.setMetricsContainer(container); + MetricsEnvironment.setCurrentContainer(container); Distribution distribution = Metrics.distribution(NS, NAME); @@ -80,7 +80,7 @@ public void distributionToCell() { @Test public void counterToCell() { MetricsContainer container = new MetricsContainer("step"); - MetricsEnvironment.setMetricsContainer(container); + MetricsEnvironment.setCurrentContainer(container); Counter counter = Metrics.counter(NS, NAME); CounterCell cell = container.getCounter(METRIC_NAME); counter.inc(); From 1cec9702e62b64252149645627d96889edfeb33e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 22 Nov 2016 14:51:39 -0800 Subject: [PATCH 008/279] Output Keyed Bundles in GroupAlsoByWindowEvaluator This allows reuse of keys for downstream serialization. --- .../java/org/apache/beam/runners/direct/DirectRunner.java | 5 ++++- .../runners/direct/GroupAlsoByWindowEvaluatorFactory.java | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 0060e847f825..cb319473ff26 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -31,6 +31,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.GBKIntoKeyedWorkItems; +import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; @@ -310,7 +311,9 @@ public DirectPipelineResult run(Pipeline pipeline) { KeyedPValueTrackingVisitor keyedPValueVisitor = KeyedPValueTrackingVisitor.create( ImmutableSet.>of( - GroupByKey.class, DirectGroupByKeyOnly.class)); + GBKIntoKeyedWorkItems.class, + DirectGroupByKeyOnly.class, + DirectGroupAlsoByWindow.class)); pipeline.traverseTopologically(keyedPValueVisitor); DisplayDataValidator.validatePipeline(pipeline); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index b946e4d76bb5..36c742b801c6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -112,6 +112,7 @@ private static class GroupAlsoByWindowEvaluator private @SuppressWarnings("unchecked") final WindowingStrategy windowingStrategy; + private final StructuralKey structuralKey; private final Collection> outputBundles; private final ImmutableList.Builder>> unprocessedElements; private final AggregatorContainer.Mutator aggregatorChanges; @@ -130,6 +131,7 @@ public GroupAlsoByWindowEvaluator( this.evaluationContext = evaluationContext; this.application = application; + structuralKey = inputBundle.getKey(); stepContext = evaluationContext .getExecutionContext(application, inputBundle.getKey()) .getOrCreateStepContext( @@ -159,7 +161,7 @@ public void processElement(WindowedValue> element) throws Ex K key = workItem.key(); UncommittedBundle>> bundle = - evaluationContext.createBundle(application.getOutput()); + evaluationContext.createKeyedBundle(structuralKey, application.getOutput()); outputBundles.add(bundle); CopyOnAccessInMemoryStateInternals stateInternals = (CopyOnAccessInMemoryStateInternals) stepContext.stateInternals(); From 73d0af9151340d85df1f720e88366f8a463b44bc Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 22 Nov 2016 16:14:29 -0800 Subject: [PATCH 009/279] Add TransformHierarchyTest This tests basic features of TransformHierarchy --- .../sdk/runners/TransformHierarchyTest.java | 142 ++++++++++++++++++ 1 file changed, 142 insertions(+) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java new file mode 100644 index 000000000000..c28f23efd607 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -0,0 +1,142 @@ +/* + * 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.runners; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +import java.util.HashSet; +import java.util.Set; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PValue; +import org.hamcrest.Matchers; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link TransformHierarchy}. + */ +@RunWith(JUnit4.class) +public class TransformHierarchyTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + private TransformHierarchy hierarchy; + private TestPipeline pipeline; + + @Before + public void setup() { + hierarchy = new TransformHierarchy(); + pipeline = TestPipeline.create(); + } + + @Test + public void getCurrentNoPushReturnsRoot() { + assertThat(hierarchy.getCurrent().isRootNode(), is(true)); + } + + @Test + public void popWithoutPushThrows() { + thrown.expect(IllegalStateException.class); + hierarchy.popNode(); + } + + @Test + public void pushThenPopSucceeds() { + TransformTreeNode root = hierarchy.getCurrent(); + TransformTreeNode node = + new TransformTreeNode(hierarchy.getCurrent(), Create.of(1), "Create", PBegin.in(pipeline)); + hierarchy.pushNode(node); + assertThat(hierarchy.getCurrent(), equalTo(node)); + hierarchy.popNode(); + assertThat(hierarchy.getCurrent(), equalTo(root)); + } + + @Test + public void visitVisitsAllPushed() { + TransformTreeNode root = hierarchy.getCurrent(); + Create.Values create = Create.of(1); + PCollection created = pipeline.apply(create); + PBegin begin = PBegin.in(pipeline); + + TransformTreeNode compositeNode = + new TransformTreeNode(root, create, "Create", begin); + root.addComposite(compositeNode); + TransformTreeNode primitiveNode = + new TransformTreeNode( + compositeNode, Read.from(CountingSource.upTo(1L)), "Create/Read", begin); + compositeNode.addComposite(primitiveNode); + + TransformTreeNode otherPrimitive = + new TransformTreeNode( + root, MapElements.via(new SimpleFunction() { + @Override + public Integer apply(Integer input) { + return input; + } + }), "ParDo", created); + root.addComposite(otherPrimitive); + otherPrimitive.addInputProducer(created, primitiveNode); + + hierarchy.pushNode(compositeNode); + hierarchy.pushNode(primitiveNode); + hierarchy.popNode(); + hierarchy.popNode(); + hierarchy.pushNode(otherPrimitive); + hierarchy.popNode(); + + final Set visitedCompositeNodes = new HashSet<>(); + final Set visitedPrimitiveNodes = new HashSet<>(); + final Set visitedValuesInVisitor = new HashSet<>(); + + Set visitedValues = new HashSet<>(); + hierarchy.visit(new PipelineVisitor.Defaults() { + @Override + public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + visitedCompositeNodes.add(node); + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void visitPrimitiveTransform(TransformTreeNode node) { + visitedPrimitiveNodes.add(node); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + visitedValuesInVisitor.add(value); + } + }, visitedValues); + + assertThat(visitedCompositeNodes, containsInAnyOrder(root, compositeNode)); + assertThat(visitedPrimitiveNodes, containsInAnyOrder(primitiveNode, otherPrimitive)); + assertThat(visitedValuesInVisitor, Matchers.containsInAnyOrder(created)); + } +} From 09986e9433d49812d5061fe6543dff90d78eba6a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 22 Nov 2016 22:16:29 -0800 Subject: [PATCH 010/279] Use more natural class to find class loader in ReflectHelpers --- .../java/org/apache/beam/sdk/util/common/ReflectHelpers.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java index 637e8e3d15ef..4ec39c1700b9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java @@ -39,7 +39,6 @@ import java.util.ServiceLoader; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import org.apache.beam.sdk.util.IOChannelUtils; /** * Utilities for working with with {@link Class Classes} and {@link Method Methods}. @@ -225,7 +224,7 @@ public static Iterable getClosureOfMethodsOnInterface(Class iface) { public static ClassLoader findClassLoader() { ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); if (classLoader == null) { - classLoader = IOChannelUtils.class.getClassLoader(); + classLoader = ReflectHelpers.class.getClassLoader(); } if (classLoader == null) { classLoader = ClassLoader.getSystemClassLoader(); From 2e2146b1869807d69658592de8ed5ff339c28507 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Tue, 22 Nov 2016 11:38:00 -0800 Subject: [PATCH 011/279] Update transitive dependencies for Apex 3.5.0 snapshot version. --- runners/apex/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index d0b0fdf1d208..84185b8ec592 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -216,11 +216,11 @@ org.apache.apex:apex-api:jar:3.5.0-SNAPSHOT org.apache.commons:commons-lang3::3.1 - commons-io:commons-io:jar:2.1 + commons-io:commons-io:jar: com.esotericsoftware.kryo:kryo::2.24.0 - com.datatorrent:netlet::1.2.1 + com.datatorrent:netlet:: org.slf4j:slf4j-api:jar:1.7.14 - org.apache.hadoop:hadoop-common:jar:2.2.0 + org.apache.hadoop:hadoop-common:jar: joda-time:joda-time:jar:2.4 com.google.guava:guava:jar:19.0 From ef74e192eaee79e4cb8c7c901a296dd76559d76d Mon Sep 17 00:00:00 2001 From: Daniel Kulp Date: Tue, 22 Nov 2016 13:31:19 -0500 Subject: [PATCH 012/279] [BEAM-1034] Clean up tmp area in tests --- .../sorter/BufferedExternalSorter.java | 6 +- .../sdk/extensions/sorter/ExternalSorter.java | 6 +- .../sorter/BufferedExternalSorterTest.java | 58 ++++++++++++++++--- .../extensions/sorter/ExternalSorterTest.java | 53 ++++++++++++++--- 4 files changed, 103 insertions(+), 20 deletions(-) diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java index 0f89e301467a..1dfd3395365a 100644 --- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java +++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java @@ -35,12 +35,13 @@ public static class Options implements Serializable { private int memoryMB = 100; /** Sets the path to a temporary location where the sorter writes intermediate files. */ - public void setTempLocation(String tempLocation) { + public Options setTempLocation(String tempLocation) { checkArgument( !tempLocation.startsWith("gs://"), "BufferedExternalSorter does not support GCS temporary location"); this.tempLocation = tempLocation; + return this; } /** Returns the configured temporary location. */ @@ -52,9 +53,10 @@ public String getTempLocation() { * Sets the size of the memory buffer in megabytes. This controls both the buffer for initial in * memory sorting and the buffer used when external sorting. Must be greater than zero. */ - public void setMemoryMB(int memoryMB) { + public Options setMemoryMB(int memoryMB) { checkArgument(memoryMB > 0, "memoryMB must be greater than zero"); this.memoryMB = memoryMB; + return this; } /** Returns the configured size of the memory buffer. */ diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java index 3cf0cc00e73a..beef1eebc4b5 100644 --- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java +++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java @@ -67,12 +67,13 @@ public static class Options implements Serializable { private int memoryMB = 100; /** Sets the path to a temporary location where the sorter writes intermediate files. */ - public void setTempLocation(String tempLocation) { + public Options setTempLocation(String tempLocation) { if (tempLocation.startsWith("gs://")) { throw new IllegalArgumentException("Sorter doesn't support GCS temporary location."); } this.tempLocation = tempLocation; + return this; } /** Returns the configured temporary location. */ @@ -81,9 +82,10 @@ public String getTempLocation() { } /** Sets the size of the memory buffer in megabytes. */ - public void setMemoryMB(int memoryMB) { + public Options setMemoryMB(int memoryMB) { checkArgument(memoryMB > 0, "memoryMB must be greater than zero"); this.memoryMB = memoryMB; + return this; } /** Returns the configured size of the memory buffer. */ diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java index 63dbedfaafb7..8c108ebf8024 100644 --- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java +++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorterTest.java @@ -27,9 +27,17 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; import java.util.Arrays; import org.apache.beam.sdk.extensions.sorter.SorterTestUtils.SorterGenerator; import org.apache.beam.sdk.values.KV; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -40,6 +48,29 @@ @RunWith(JUnit4.class) public class BufferedExternalSorterTest { @Rule public ExpectedException thrown = ExpectedException.none(); + static Path tmpLocation; + + @BeforeClass + public static void setupTempDir() throws IOException { + tmpLocation = Files.createTempDirectory("tmp"); + } + + @AfterClass + public static void cleanupTempDir() throws IOException { + Files.walkFileTree(tmpLocation, new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + Files.delete(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + Files.delete(dir); + return FileVisitResult.CONTINUE; + } + }); + } @SuppressWarnings("unchecked") @Test @@ -106,25 +137,29 @@ public void testFallback() throws Exception { @Test public void testEmpty() throws Exception { - SorterTestUtils.testEmpty(BufferedExternalSorter.create(new BufferedExternalSorter.Options())); + SorterTestUtils.testEmpty(BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test public void testSingleElement() throws Exception { SorterTestUtils.testSingleElement( - BufferedExternalSorter.create(new BufferedExternalSorter.Options())); + BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test public void testEmptyKeyValueElement() throws Exception { SorterTestUtils.testEmptyKeyValueElement( - BufferedExternalSorter.create(new BufferedExternalSorter.Options())); + BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test public void testMultipleIterations() throws Exception { SorterTestUtils.testMultipleIterations( - BufferedExternalSorter.create(new BufferedExternalSorter.Options())); + BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test @@ -133,7 +168,8 @@ public void testManySortersFewRecords() throws Exception { new SorterGenerator() { @Override public Sorter generateSorter() throws Exception { - return BufferedExternalSorter.create(new BufferedExternalSorter.Options()); + return BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString())); } }, 1000000, @@ -146,7 +182,8 @@ public void testOneSorterManyRecords() throws Exception { new SorterGenerator() { @Override public Sorter generateSorter() throws Exception { - return BufferedExternalSorter.create(new BufferedExternalSorter.Options()); + return BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString())); } }, 1, @@ -156,14 +193,16 @@ public Sorter generateSorter() throws Exception { @Test public void testAddAfterSort() throws Exception { SorterTestUtils.testAddAfterSort( - BufferedExternalSorter.create(new BufferedExternalSorter.Options()), thrown); + BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString())), thrown); fail(); } @Test public void testSortTwice() throws Exception { SorterTestUtils.testSortTwice( - BufferedExternalSorter.create(new BufferedExternalSorter.Options()), thrown); + BufferedExternalSorter.create(new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString())), thrown); fail(); } @@ -171,7 +210,8 @@ public void testSortTwice() throws Exception { public void testNegativeMemory() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("memoryMB must be greater than zero"); - BufferedExternalSorter.Options options = new BufferedExternalSorter.Options(); + BufferedExternalSorter.Options options = new BufferedExternalSorter.Options() + .setTempLocation(tmpLocation.toString()); options.setMemoryMB(-1); } } diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java index 9232b6229957..bcfbdade94a7 100644 --- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java +++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java @@ -20,7 +20,16 @@ import static org.junit.Assert.fail; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; + import org.apache.beam.sdk.extensions.sorter.SorterTestUtils.SorterGenerator; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -31,25 +40,52 @@ @RunWith(JUnit4.class) public class ExternalSorterTest { @Rule public ExpectedException thrown = ExpectedException.none(); + static Path tmpLocation; + + @BeforeClass + public static void setupTempDir() throws IOException { + tmpLocation = Files.createTempDirectory("tmp"); + } + + @AfterClass + public static void cleanupTempDir() throws IOException { + Files.walkFileTree(tmpLocation, new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + Files.delete(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + Files.delete(dir); + return FileVisitResult.CONTINUE; + } + }); + } @Test public void testEmpty() throws Exception { - SorterTestUtils.testEmpty(ExternalSorter.create(new ExternalSorter.Options())); + SorterTestUtils.testEmpty(ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test public void testSingleElement() throws Exception { - SorterTestUtils.testSingleElement(ExternalSorter.create(new ExternalSorter.Options())); + SorterTestUtils.testSingleElement(ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test public void testEmptyKeyValueElement() throws Exception { - SorterTestUtils.testEmptyKeyValueElement(ExternalSorter.create(new ExternalSorter.Options())); + SorterTestUtils.testEmptyKeyValueElement(ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test public void testMultipleIterations() throws Exception { - SorterTestUtils.testMultipleIterations(ExternalSorter.create(new ExternalSorter.Options())); + SorterTestUtils.testMultipleIterations(ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString()))); } @Test @@ -58,7 +94,8 @@ public void testRandom() throws Exception { new SorterGenerator() { @Override public Sorter generateSorter() throws Exception { - return ExternalSorter.create(new ExternalSorter.Options()); + return ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString())); } }, 1, @@ -67,13 +104,15 @@ public Sorter generateSorter() throws Exception { @Test public void testAddAfterSort() throws Exception { - SorterTestUtils.testAddAfterSort(ExternalSorter.create(new ExternalSorter.Options()), thrown); + SorterTestUtils.testAddAfterSort(ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString())), thrown); fail(); } @Test public void testSortTwice() throws Exception { - SorterTestUtils.testSortTwice(ExternalSorter.create(new ExternalSorter.Options()), thrown); + SorterTestUtils.testSortTwice(ExternalSorter.create(new ExternalSorter.Options() + .setTempLocation(tmpLocation.toString())), thrown); fail(); } From 7b314aad1c7c62ad61e09e610c60f53ac056d75d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 17 Nov 2016 17:07:21 +0100 Subject: [PATCH 013/279] [BEAM-959] Improve validation messages in JdbcIO --- .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 52 +++++++++++++------ 1 file changed, 35 insertions(+), 17 deletions(-) diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index 0e0703fbd63e..9644a65f96ab 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.io.jdbc; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; @@ -183,16 +183,20 @@ abstract static class Builder { } public static DataSourceConfiguration create(DataSource dataSource) { - checkNotNull(dataSource, "dataSource"); - checkArgument(dataSource instanceof Serializable, "dataSource must be Serializable"); + checkArgument(dataSource != null, "DataSourceConfiguration.create(dataSource) called with " + + "null data source"); + checkArgument(dataSource instanceof Serializable, + "DataSourceConfiguration.create(dataSource) called with a dataSource not Serializable"); return new AutoValue_JdbcIO_DataSourceConfiguration.Builder() .setDataSource(dataSource) .build(); } public static DataSourceConfiguration create(String driverClassName, String url) { - checkNotNull(driverClassName, "driverClassName"); - checkNotNull(url, "url"); + checkArgument(driverClassName != null, + "DataSourceConfiguration.create(driverClassName, url) called with null driverClassName"); + checkArgument(url != null, + "DataSourceConfiguration.create(driverClassName, url) called with null url"); return new AutoValue_JdbcIO_DataSourceConfiguration.Builder() .setDriverClassName(driverClassName) .setUrl(url) @@ -263,27 +267,31 @@ abstract static class Builder { } public Read withDataSourceConfiguration(DataSourceConfiguration configuration) { - checkNotNull(configuration, "configuration"); + checkArgument(configuration != null, "JdbcIO.read().withDataSourceConfiguration" + + "(configuration) called with null configuration"); return toBuilder().setDataSourceConfiguration(configuration).build(); } public Read withQuery(String query) { - checkNotNull(query, "query"); + checkArgument(query != null, "JdbcIO.read().withQuery(query) called with null query"); return toBuilder().setQuery(query).build(); } public Read withStatementPrepator(StatementPreparator statementPreparator) { - checkNotNull(statementPreparator, "statementPreparator"); + checkArgument(statementPreparator != null, + "JdbcIO.read().withStatementPreparator(statementPreparator) called " + + "with null statementPreparator"); return toBuilder().setStatementPreparator(statementPreparator).build(); } public Read withRowMapper(RowMapper rowMapper) { - checkNotNull(rowMapper, "rowMapper"); + checkArgument(rowMapper != null, + "JdbcIO.read().withRowMapper(rowMapper) called with null rowMapper"); return toBuilder().setRowMapper(rowMapper).build(); } public Read withCoder(Coder coder) { - checkNotNull(coder, "coder"); + checkArgument(coder != null, "JdbcIO.read().withCoder(coder) called with null coder"); return toBuilder().setCoder(coder).build(); } @@ -314,10 +322,15 @@ public void processElement(ProcessContext context) { @Override public void validate(PBegin input) { - checkNotNull(getQuery(), "query"); - checkNotNull(getRowMapper(), "rowMapper"); - checkNotNull(getCoder(), "coder"); - checkNotNull(getDataSourceConfiguration()); + checkState(getQuery() != null, + "JdbcIO.read() requires a query to be set via withQuery(query)"); + checkState(getRowMapper() != null, + "JdbcIO.read() requires a rowMapper to be set via withRowMapper(rowMapper)"); + checkState(getCoder() != null, + "JdbcIO.read() requires a coder to be set via withCoder(coder)"); + checkState(getDataSourceConfiguration() != null, + "JdbcIO.read() requires a DataSource configuration to be set via " + + "withDataSourceConfiguration(dataSourceConfiguration)"); } @Override @@ -411,9 +424,14 @@ public PDone apply(PCollection input) { @Override public void validate(PCollection input) { - checkNotNull(getDataSourceConfiguration(), "dataSourceConfiguration"); - checkNotNull(getStatement(), "statement"); - checkNotNull(getPreparedStatementSetter(), "preparedStatementSetter"); + checkArgument(getDataSourceConfiguration() != null, + "JdbcIO.write() requires a configuration to be set via " + + ".withDataSourceConfiguration(configuration)"); + checkArgument(getStatement() != null, + "JdbcIO.write() requires a statement to be set via .withStatement(statement)"); + checkArgument(getPreparedStatementSetter() != null, + "JdbcIO.write() requires a preparedStatementSetter to be set via " + + ".withPreparedStatementSetter(preparedStatementSetter)"); } private static class WriteFn extends DoFn { From 4a097729ac9fc65283f4f11f85812188589c8df3 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 8 Nov 2016 11:03:21 +0100 Subject: [PATCH 014/279] Replace WindowAssignment OldDoFn by FlatMap in Flink Runner The streaming runner had an OldDoFn that was used for assigning windows using a WindowFn. This is now done with a FlatMap. --- .../FlinkStreamingTransformTranslators.java | 63 +++---------------- 1 file changed, 9 insertions(+), 54 deletions(-) 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 40dfbb990b50..47935eb3d6ae 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.nio.ByteBuffer; @@ -31,6 +30,7 @@ import java.util.Set; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.flink.FlinkRunner; +import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; 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.streaming.DoFnOperator; @@ -53,7 +53,6 @@ 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.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.RawUnionValue; @@ -637,64 +636,20 @@ public void translateNode( TypeInformation> typeInfo = context.getTypeInfo(context.getOutput(transform)); - OldDoFn windowAssignerDoFn = - createWindowAssigner(windowingStrategy.getWindowFn()); - - @SuppressWarnings("unchecked") - PCollection inputPCollection = context.getInput(transform); - - TypeInformation> inputTypeInfo = - context.getTypeInfo(inputPCollection); - - DoFnOperator> doFnOperator = new DoFnOperator<>( - windowAssignerDoFn, - inputTypeInfo, - new TupleTag("main output"), - Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory>(), - windowingStrategy, - new HashMap>(), /* side-input mapping */ - Collections.>emptyList(), /* side inputs */ - context.getPipelineOptions()); - DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); - SingleOutputStreamOperator> outDataStream = inputDataStream - .transform(transform.getName(), typeInfo, doFnOperator); - - context.setOutputDataStream(context.getOutput(transform), outDataStream); - } + WindowFn windowFn = windowingStrategy.getWindowFn(); - private static OldDoFn createWindowAssigner( - final WindowFn windowFn) { + FlinkAssignWindows assignWindowsFunction = + new FlinkAssignWindows<>(windowFn); - return new OldDoFn() { + SingleOutputStreamOperator> outputDataStream = inputDataStream + .flatMap(assignWindowsFunction) + .name(context.getOutput(transform).getName()) + .returns(typeInfo); - @Override - public void processElement(final ProcessContext c) throws Exception { - Collection windows = windowFn.assignWindows( - windowFn.new AssignContext() { - @Override - public T element() { - return c.element(); - } - - @Override - public Instant timestamp() { - return c.timestamp(); - } - - @Override - public BoundedWindow window() { - return Iterables.getOnlyElement(c.windowingInternals().windows()); - } - }); - - c.windowingInternals().outputWindowedValue( - c.element(), c.timestamp(), windows, c.pane()); - } - }; + context.setOutputDataStream(context.getOutput(transform), outputDataStream); } } From 8d7d46c6e407c738a61b236078d002d178da0b9f Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 23 Nov 2016 09:24:05 +0800 Subject: [PATCH 015/279] [BEAM-800] add getFn to DoFnInvoker --- .../sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java | 5 +++++ .../org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java | 3 +++ .../org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java | 5 +++++ 3 files changed, 13 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java index 9998c9d78801..408ea98e2d51 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java @@ -175,6 +175,11 @@ public void invokeOnTimer( timerId, delegate.getClass().getName())); } } + + @Override + public DoFn getFn() { + return delegate; + } } /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index d899207a6f34..5e61bddbe4d9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -84,6 +84,9 @@ void invokeSplitRestriction( > TrackerT invokeNewTracker( RestrictionT restriction); + /** Get the bound {@link DoFn}. */ + DoFn getFn(); + /** * Interface for runner implementors to provide implementations of extra context information. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index 15ba1987eaab..4ad7dad30e95 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -180,5 +180,10 @@ public void invokeSplitRestriction( TrackerT invokeNewTracker(RestrictionT restriction) { throw new UnsupportedOperationException("OldDoFn is not splittable"); } + + @Override + public DoFn getFn() { + throw new UnsupportedOperationException("getFn is not supported for OldDoFn"); + } } } From 07544ef3a47bbdfacc00c75af875c3533a5fe477 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 23 Nov 2016 11:22:08 -0800 Subject: [PATCH 016/279] Remove unused body of StreamingPCollectionViewWriterFn --- .../beam/runners/dataflow/DataflowRunner.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 36328e9acf92..f1d41f235247 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -2360,8 +2360,8 @@ public final PCollection apply(PBegin input) { } /** - * A specialized {@link DoFn} for writing the contents of a {@link PCollection} - * to a streaming {@link PCollectionView} backend implementation. + * A marker {@link DoFn} for writing the contents of a {@link PCollection} to a streaming + * {@link PCollectionView} backend implementation. */ @Deprecated public static class StreamingPCollectionViewWriterFn @@ -2389,13 +2389,9 @@ public Coder getDataCoder() { @Override public void processElement(ProcessContext c) throws Exception { - List> output = new ArrayList<>(); - for (T elem : c.element()) { - output.add(WindowedValue.of(elem, c.timestamp(), c.window(), c.pane())); - } - - c.windowingInternals().writePCollectionViewData( - view.getTagInternal(), output, dataCoder); + throw new UnsupportedOperationException( + String.format( + "%s is a marker class only and should never be executed.", getClass().getName())); } } From 803bbe2a3026424f509e13809a8eecb79990e5fe Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 23 Nov 2016 11:23:07 -0800 Subject: [PATCH 017/279] Remove unused WindowingInternals.writePCollectionViewData --- .../operators/ApexGroupByKeyOperator.java | 10 ---------- .../beam/runners/core/SimpleDoFnRunner.java | 18 ------------------ .../beam/runners/core/SimpleOldDoFnRunner.java | 16 ---------------- .../functions/FlinkProcessContextBase.java | 8 -------- .../spark/translation/SparkProcessContext.java | 9 --------- .../apache/beam/sdk/transforms/DoFnTester.java | 10 ---------- .../beam/sdk/util/WindowingInternals.java | 10 ---------- 7 files changed, 81 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index eca430804f28..3b0e4f283341 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -30,8 +30,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; - -import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; @@ -40,7 +38,6 @@ import java.util.Iterator; import java.util.Map; import java.util.Set; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; @@ -391,13 +388,6 @@ public PaneInfo pane() { throw new UnsupportedOperationException("pane() is not available in Streaming mode."); } - @Override - public void writePCollectionViewData( - TupleTag tag, Iterable> data, Coder elemCoder) - throws IOException { - throw new RuntimeException("writePCollectionViewData() not available in Streaming mode."); - } - @Override public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { throw new RuntimeException("sideInput() is not available in Streaming mode."); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 841e4124b5cc..f611c0af572f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -21,14 +21,11 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; @@ -594,21 +591,6 @@ public TimerInternals timerInternals() { return context.stepContext.timerInternals(); } - @Override - public void writePCollectionViewData( - TupleTag tag, Iterable> data, Coder elemCoder) - throws IOException { - @SuppressWarnings("unchecked") - Coder windowCoder = (Coder) context.windowFn.windowCoder(); - - context.stepContext.writePCollectionViewData( - tag, - data, - IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)), - window(), - windowCoder); - } - @Override public StateInternals stateInternals() { return context.stepContext.stateInternals(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index cbda791ebcc6..73286ade209b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -21,14 +21,11 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; @@ -495,19 +492,6 @@ public TimerInternals timerInternals() { return context.stepContext.timerInternals(); } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder elemCoder) throws IOException { - @SuppressWarnings("unchecked") - Coder windowCoder = (Coder) context.windowFn.windowCoder(); - - context.stepContext.writePCollectionViewData( - tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)), - window(), windowCoder); - } - @Override public StateInternals stateInternals() { return context.stepContext.stateInternals(); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java index 21697859d381..42607dd0b910 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java @@ -20,14 +20,12 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.collect.Iterables; -import java.io.IOException; import java.io.Serializable; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.Map; 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; @@ -152,12 +150,6 @@ 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, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index 6a6cbd43d3c8..bb0ec2f6c3ec 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -23,14 +23,12 @@ import com.google.common.collect.AbstractIterator; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.Map; import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.runners.spark.util.SparkSideInputReader; -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; @@ -281,13 +279,6 @@ public PaneInfo pane() { return windowedValue.getPane(); } - @Override - public void writePCollectionViewData( - TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - throw new UnsupportedOperationException( - "WindowingInternals#writePCollectionViewData() is not yet supported."); - } - @Override public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { throw new UnsupportedOperationException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index bbf031542d71..daa8a0600e8a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -34,7 +33,6 @@ import java.util.List; import java.util.Map; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -700,14 +698,6 @@ public PaneInfo pane() { return element.getPane(); } - @Override - public void writePCollectionViewData( - TupleTag tag, Iterable> data, Coder elemCoder) - throws IOException { - throw new UnsupportedOperationException( - "WritePCollectionViewData is not supported in in the context of DoFnTester"); - } - @Override public T sideInput( PCollectionView view, BoundedWindow sideInputWindow) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java index 5e908646b99a..a92172565755 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java @@ -17,9 +17,7 @@ */ package org.apache.beam.sdk.util; -import java.io.IOException; import java.util.Collection; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -77,14 +75,6 @@ void sideOutputWindowedValue( */ PaneInfo pane(); - /** - * Write the given {@link PCollectionView} data to a location accessible by other workers. - */ - void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder elemCoder) throws IOException; - /** * Return the value of the side input for a particular side input window. */ From cc96b1381b6db849adf69daddecf30b9c61acf73 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Fri, 25 Nov 2016 14:52:26 +0100 Subject: [PATCH 018/279] [BEAM-851] Determine if the pipeline must be translated into streaming mode (if not set) Now an Evaluator (visitor) detects if there are Unbonded.Read transforms. This approach is based on Flink's PipelineTranslationOptimizer --- .../beam/runners/spark/SparkRunner.java | 61 ++++++++++++++++++- .../StreamingTransformTranslator.java | 2 +- .../streaming/EmptyStreamAssertionTest.java | 2 + .../streaming/FlattenStreamingTest.java | 2 + .../SimpleStreamingWordCountTest.java | 1 + .../SparkTestPipelineOptionsForStreaming.java | 6 -- 6 files changed, 65 insertions(+), 9 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index e800071edf12..49e0113159aa 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -27,6 +27,7 @@ import org.apache.beam.runners.spark.translation.TransformTranslator; import org.apache.beam.runners.spark.translation.streaming.SparkRunnerStreamingContextFactory; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PipelineOptionsValidator; @@ -120,12 +121,12 @@ private SparkRunner(SparkPipelineOptions options) { mOptions = options; } - @Override public EvaluationResult run(Pipeline pipeline) { try { LOG.info("Executing pipeline using the SparkRunner."); + detectTranslationMode(pipeline); if (mOptions.isStreaming()) { SparkRunnerStreamingContextFactory contextFactory = new SparkRunnerStreamingContextFactory(pipeline, mOptions); @@ -136,7 +137,7 @@ public EvaluationResult run(Pipeline pipeline) { jssc.start(); // if recovering from checkpoint, we have to reconstruct the EvaluationResult instance. - return contextFactory.getCtxt() == null ? new EvaluationContext(jssc.sc(), + return contextFactory.getCtxt() == null ? new EvaluationContext(jssc.sparkContext(), pipeline, jssc) : contextFactory.getCtxt(); } else { JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions); @@ -167,6 +168,62 @@ public EvaluationResult run(Pipeline pipeline) { } } + /** + * Detect the translation mode for the pipeline and change options in case streaming + * translation is needed. + * @param pipeline + */ + private void detectTranslationMode(Pipeline pipeline) { + TranslationModeDetector detector = new TranslationModeDetector(); + pipeline.traverseTopologically(detector); + if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) { + // set streaming mode if it's a streaming pipeline + this.mOptions.setStreaming(true); + } + } + + /** + * The translation mode of the Beam Pipeline. + */ + enum TranslationMode { + /** Uses the batch mode. */ + BATCH, + /** Uses the streaming mode. */ + STREAMING + } + + /** + * Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. + */ + static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults { + private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class); + + private TranslationMode translationMode; + + TranslationModeDetector(TranslationMode defaultMode) { + this.translationMode = defaultMode; + } + + TranslationModeDetector() { + this(TranslationMode.BATCH); + } + + TranslationMode getTranslationMode() { + return translationMode; + } + + @Override + public void visitPrimitiveTransform(TransformTreeNode node) { + if (translationMode.equals(TranslationMode.BATCH)) { + Class transformClass = node.getTransform().getClass(); + if (transformClass == Read.Unbounded.class) { + LOG.info("Found {}. Switching to streaming execution.", transformClass); + translationMode = TranslationMode.STREAMING; + } + } + } + } + /** * Evaluator on the pipeline. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index b30f0793135c..6ed5b559e853 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -466,7 +466,7 @@ public boolean hasTranslation(Class> clazz) { @SuppressWarnings("unchecked") TransformEvaluator transformEvaluator = (TransformEvaluator) EVALUATORS.get(clazz); checkState(transformEvaluator != null, - "No TransformEvaluator registered for for UNBOUNDED transform %s", clazz); + "No TransformEvaluator registered for UNBOUNDED transform %s", clazz); return transformEvaluator; } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java index d40bcfff29c0..656107a7ea27 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java @@ -57,6 +57,8 @@ public class EmptyStreamAssertionTest implements Serializable { @Test public void testAssertion() throws Exception { SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir); + options.setStreaming(true); + Duration windowDuration = new Duration(options.getBatchIntervalMillis()); Pipeline pipeline = Pipeline.create(options); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java index 3e75b1834cc1..d36796a5a65f 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/FlattenStreamingTest.java @@ -63,6 +63,7 @@ public class FlattenStreamingTest { @Test public void testFlattenUnbounded() throws Exception { SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir); + options.setStreaming(true); Pipeline p = Pipeline.create(options); PCollection w1 = @@ -82,6 +83,7 @@ public void testFlattenUnbounded() throws Exception { @Test public void testFlattenBoundedUnbounded() throws Exception { SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir); + options.setStreaming(true); Pipeline p = Pipeline.create(options); PCollection w1 = diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java index 9a15ff293446..3734cf6d38ca 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SimpleStreamingWordCountTest.java @@ -64,6 +64,7 @@ public class SimpleStreamingWordCountTest implements Serializable { @Test public void testFixedWindows() throws Exception { SparkPipelineOptions options = pipelineOptions.withTmpCheckpointDir(checkpointParentDir); + options.setStreaming(true); // override defaults options.setBatchIntervalMillis(BATCH_INTERVAL.getMillis()); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/SparkTestPipelineOptionsForStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/SparkTestPipelineOptionsForStreaming.java index f74c74ab47e9..28f6d5dd6dd3 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/SparkTestPipelineOptionsForStreaming.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/SparkTestPipelineOptionsForStreaming.java @@ -28,12 +28,6 @@ */ public class SparkTestPipelineOptionsForStreaming extends SparkTestPipelineOptions { - @Override - protected void before() throws Throwable { - super.before(); - options.setStreaming(true); - } - public SparkPipelineOptions withTmpCheckpointDir(TemporaryFolder parent) throws IOException { // tests use JUnit's TemporaryFolder path in the form of: /.../junit/... From f6005593ccf6bdef7c975622d5af39792c7db44c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 27 Nov 2016 11:39:08 +0100 Subject: [PATCH 019/279] [BEAM-1049] Update spark version to 1.6.3 --- runners/spark/pom.xml | 2 +- .../beam/runners/spark/aggregators/AccumulatorSingleton.java | 2 +- .../streaming/SparkRunnerStreamingContextFactory.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 88223e2b49c0..da7a72a37a5b 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -34,7 +34,7 @@ UTF-8 UTF-8 - 1.6.2 + 1.6.3 2.2.0 0.9.0.1 3.1.2 diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java index 758372eff762..bc7105fc93ff 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java @@ -24,7 +24,7 @@ /** * For resilience, {@link Accumulator}s are required to be wrapped in a Singleton. - * @see accumulators + * @see accumulators */ public class AccumulatorSingleton { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java index af90ff169bb6..d069a1121eb3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java @@ -40,7 +40,7 @@ /** * A {@link JavaStreamingContext} factory for resilience. - * @see how-to-configure-checkpointing + * @see how-to-configure-checkpointing */ public class SparkRunnerStreamingContextFactory implements JavaStreamingContextFactory { private static final Logger LOG = From 7502adda3262bce9d6d4fe4499bde8d8b5273029 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 22 Nov 2016 16:01:45 -0800 Subject: [PATCH 020/279] Add input type to TransformResult This would likely have caught some hard-to-diagnose type safety errors during the development of StatefulParDoEvaluatorFactory, so adding it should hopefully catch similar bugs in the future. --- .../direct/AbstractModelEnforcement.java | 2 +- .../direct/BoundedReadEvaluatorFactory.java | 2 +- .../beam/runners/direct/CommittedResult.java | 2 +- .../runners/direct/CompletionCallback.java | 2 +- ...ycleManagerRemovingTransformEvaluator.java | 2 +- .../direct/EmptyTransformEvaluator.java | 4 +- .../runners/direct/EvaluationContext.java | 2 +- .../ExecutorServiceParallelExecutor.java | 2 +- .../direct/FlattenEvaluatorFactory.java | 10 ++--- .../GroupAlsoByWindowEvaluatorFactory.java | 5 ++- .../GroupByKeyOnlyEvaluatorFactory.java | 2 +- .../ImmutabilityEnforcementFactory.java | 2 +- .../beam/runners/direct/ModelEnforcement.java | 2 +- .../beam/runners/direct/ParDoEvaluator.java | 2 +- .../direct/PassthroughTransformEvaluator.java | 4 +- .../runners/direct/StepTransformResult.java | 38 +++++++++------- .../direct/TestStreamEvaluatorFactory.java | 2 +- .../runners/direct/TransformEvaluator.java | 2 +- .../runners/direct/TransformExecutor.java | 4 +- .../beam/runners/direct/TransformResult.java | 16 +++++-- .../direct/UnboundedReadEvaluatorFactory.java | 3 +- .../runners/direct/ViewEvaluatorFactory.java | 2 +- .../direct/WindowEvaluatorFactory.java | 6 ++- .../BoundedReadEvaluatorFactoryTest.java | 10 ++--- ...ManagerRemovingTransformEvaluatorTest.java | 4 +- .../runners/direct/EvaluationContextTest.java | 20 ++++----- .../direct/FlattenEvaluatorFactoryTest.java | 6 +-- .../ImmutabilityEnforcementFactoryTest.java | 6 +-- .../runners/direct/ParDoEvaluatorTest.java | 2 +- .../direct/StepTransformResultTest.java | 25 ++++++----- .../TestStreamEvaluatorFactoryTest.java | 10 ++--- .../runners/direct/TransformExecutorTest.java | 45 +++++++++---------- .../UnboundedReadEvaluatorFactoryTest.java | 20 ++++++--- .../direct/WindowEvaluatorFactoryTest.java | 12 ++--- 34 files changed, 152 insertions(+), 126 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java index 81f0f5f06e1a..f09164b6fa55 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java @@ -33,6 +33,6 @@ public void afterElement(WindowedValue element) {} @Override public void afterFinish( CommittedBundle input, - TransformResult result, + TransformResult result, Iterable> outputs) {} } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 66c55cd3e00d..65b622f0b382 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -161,7 +161,7 @@ private Future> startDynamicSplitThread( } @Override - public TransformResult finishBundle() { + public TransformResult> finishBundle() { return resultBuilder.build(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java index 5fcf7b313b7f..4db7e18bbba2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java @@ -59,7 +59,7 @@ abstract class CommittedResult { public abstract Set getProducedOutputTypes(); public static CommittedResult create( - TransformResult original, + TransformResult original, CommittedBundle unprocessedElements, Iterable> outputs, Set producedOutputs) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java index 2986df13fb68..766259dfc0e1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java @@ -28,7 +28,7 @@ interface CompletionCallback { * Handle a successful result, returning the committed outputs of the result. */ CommittedResult handleResult( - CommittedBundle inputBundle, TransformResult result); + CommittedBundle inputBundle, TransformResult result); /** * Handle an input bundle that did not require processing. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index faa06151c31b..fb13b0fb2a0a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -54,7 +54,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { try { return underlying.finishBundle(); } catch (Exception e) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java index 778c5aa0f03f..85e5e702916f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyTransformEvaluator.java @@ -43,8 +43,8 @@ private EmptyTransformEvaluator(AppliedPTransform transform) { public void processElement(WindowedValue element) throws Exception {} @Override - public TransformResult finishBundle() throws Exception { - return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE) + public TransformResult finishBundle() throws Exception { + return StepTransformResult.withHold(transform, BoundedWindow.TIMESTAMP_MIN_VALUE) .build(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index b814defabcb8..c1225f63377d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -161,7 +161,7 @@ public void initialize( public CommittedResult handleResult( @Nullable CommittedBundle completedBundle, Iterable completedTimers, - TransformResult result) { + TransformResult result) { Iterable> committedBundles = commitBundles(result.getOutputBundles()); metrics.commitLogical(completedBundle, result.getLogicalMetricUpdates()); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 05cdd34922a7..b7908c5ed45d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -270,7 +270,7 @@ protected TimerIterableCompletionCallback(Iterable timers) { @Override public final CommittedResult handleResult( - CommittedBundle inputBundle, TransformResult result) { + CommittedBundle inputBundle, TransformResult result) { CommittedResult committedResult = evaluationContext.handleResult(inputBundle, timers, result); for (CommittedBundle outputBundle : committedResult.getOutputs()) { allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java index 57d56283a6e8..817e736e6b2e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java @@ -56,17 +56,17 @@ private TransformEvaluator createInMemoryEvaluator( application) { final UncommittedBundle outputBundle = evaluationContext.createBundle(application.getOutput()); - final TransformResult result = - StepTransformResult.withoutHold(application).addOutput(outputBundle).build(); + final TransformResult result = + StepTransformResult.withoutHold(application).addOutput(outputBundle).build(); return new FlattenEvaluator<>(outputBundle, result); } private static class FlattenEvaluator implements TransformEvaluator { private final UncommittedBundle outputBundle; - private final TransformResult result; + private final TransformResult result; public FlattenEvaluator( - UncommittedBundle outputBundle, TransformResult result) { + UncommittedBundle outputBundle, TransformResult result) { this.outputBundle = outputBundle; this.result = result; } @@ -77,7 +77,7 @@ public void processElement(WindowedValue element) { } @Override - public TransformResult finishBundle() { + public TransformResult finishBundle() { return result; } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 36c742b801c6..9d25bc6a8a7b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -208,10 +208,11 @@ public boolean isEmpty() { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult> finishBundle() throws Exception { // State is initialized within the constructor. It can never be null. CopyOnAccessInMemoryStateInternals state = stepContext.commitState(); - return StepTransformResult.withHold(application, state.getEarliestWatermarkHold()) + return StepTransformResult.>withHold( + application, state.getEarliestWatermarkHold()) .withState(state) .addOutput(outputBundles) .withTimerUpdate(stepContext.getTimerUpdate()) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index 0fa7ebd97464..4d691ea6dcf3 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -143,7 +143,7 @@ public void processElement(WindowedValue> element) { } @Override - public TransformResult finishBundle() { + public TransformResult> finishBundle() { Builder resultBuilder = StepTransformResult.withoutHold(application); for (Map.Entry, List>> groupedEntry : groupingMap.entrySet()) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java index 612922acfaf0..85fc374a0144 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java @@ -74,7 +74,7 @@ public void afterElement(WindowedValue element) { @Override public void afterFinish( CommittedBundle input, - TransformResult result, + TransformResult result, Iterable> outputs) { for (MutationDetector detector : mutationElements.values()) { verifyUnmodified(detector); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java index 074619aa1e72..25226f7b98e6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java @@ -58,6 +58,6 @@ public interface ModelEnforcement { */ void afterFinish( CommittedBundle input, - TransformResult result, + TransformResult result, Iterable> outputs); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 6f91319ba0a8..254fa4421b5c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -122,7 +122,7 @@ public void processElement(WindowedValue element) { } @Override - public TransformResult finishBundle() { + public TransformResult finishBundle() { try { fnRunner.finishBundle(); } catch (Exception e) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java index c6e10e542047..153af65cde4f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java @@ -42,8 +42,8 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { - return StepTransformResult.withoutHold(transform).addOutput(output).build(); + public TransformResult finishBundle() throws Exception { + return StepTransformResult.withoutHold(transform).addOutput(output).build(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java index 5719e44b0599..d58b027dd8a1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java @@ -37,18 +37,20 @@ * An immutable {@link TransformResult}. */ @AutoValue -public abstract class StepTransformResult implements TransformResult { +public abstract class StepTransformResult implements TransformResult { - public static Builder withHold(AppliedPTransform transform, Instant watermarkHold) { + public static Builder withHold( + AppliedPTransform transform, Instant watermarkHold) { return new Builder(transform, watermarkHold); } - public static Builder withoutHold(AppliedPTransform transform) { + public static Builder withoutHold( + AppliedPTransform transform) { return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE); } @Override - public TransformResult withLogicalMetricUpdates(MetricUpdates metricUpdates) { + public TransformResult withLogicalMetricUpdates(MetricUpdates metricUpdates) { return new AutoValue_StepTransformResult( getTransform(), getOutputBundles(), @@ -64,10 +66,10 @@ public TransformResult withLogicalMetricUpdates(MetricUpdates metricUpdates) { /** * A builder for creating instances of {@link StepTransformResult}. */ - public static class Builder { + public static class Builder { private final AppliedPTransform transform; private final ImmutableList.Builder> bundlesBuilder; - private final ImmutableList.Builder> unprocessedElementsBuilder; + private final ImmutableList.Builder> unprocessedElementsBuilder; private MetricUpdates metricUpdates; private CopyOnAccessInMemoryStateInternals state; private TimerUpdate timerUpdate; @@ -85,8 +87,8 @@ private Builder(AppliedPTransform transform, Instant watermarkHold) { this.metricUpdates = MetricUpdates.EMPTY; } - public StepTransformResult build() { - return new AutoValue_StepTransformResult( + public StepTransformResult build() { + return new AutoValue_StepTransformResult<>( transform, bundlesBuilder.build(), unprocessedElementsBuilder.build(), @@ -98,49 +100,51 @@ public StepTransformResult build() { producedOutputs); } - public Builder withAggregatorChanges(AggregatorContainer.Mutator aggregatorChanges) { + public Builder withAggregatorChanges(AggregatorContainer.Mutator aggregatorChanges) { this.aggregatorChanges = aggregatorChanges; return this; } - public Builder withMetricUpdates(MetricUpdates metricUpdates) { + public Builder withMetricUpdates(MetricUpdates metricUpdates) { this.metricUpdates = metricUpdates; return this; } - public Builder withState(CopyOnAccessInMemoryStateInternals state) { + public Builder withState(CopyOnAccessInMemoryStateInternals state) { this.state = state; return this; } - public Builder withTimerUpdate(TimerUpdate timerUpdate) { + public Builder withTimerUpdate(TimerUpdate timerUpdate) { this.timerUpdate = timerUpdate; return this; } - public Builder addUnprocessedElements(WindowedValue... unprocessed) { + public Builder addUnprocessedElements(WindowedValue... unprocessed) { unprocessedElementsBuilder.addAll(Arrays.asList(unprocessed)); return this; } - public Builder addUnprocessedElements(Iterable> unprocessed) { + public Builder addUnprocessedElements( + Iterable> unprocessed) { unprocessedElementsBuilder.addAll(unprocessed); return this; } - public Builder addOutput( + public Builder addOutput( UncommittedBundle outputBundle, UncommittedBundle... outputBundles) { bundlesBuilder.add(outputBundle); bundlesBuilder.add(outputBundles); return this; } - public Builder addOutput(Collection> outputBundles) { + public Builder addOutput( + Collection> outputBundles) { bundlesBuilder.addAll(outputBundles); return this; } - public Builder withAdditionalOutput(OutputType producedAdditionalOutput) { + public Builder withAdditionalOutput(OutputType producedAdditionalOutput) { producedOutputs.add(producedAdditionalOutput); return this; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 2ab6adf9528f..9df7cdc20c32 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -127,7 +127,7 @@ public void processElement(WindowedValue> element) throws Exc } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult> finishBundle() throws Exception { return resultBuilder.build(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java index 1624fcbb90b3..79c942b62df8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java @@ -42,5 +42,5 @@ public interface TransformEvaluator { * * @return an {@link TransformResult} containing the results of this bundle evaluation. */ - TransformResult finishBundle() throws Exception; + TransformResult finishBundle() throws Exception; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java index fb31cc97f74d..bbc0aaec3c90 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java @@ -159,11 +159,11 @@ private void processElements( * @return the {@link TransformResult} produced by * {@link TransformEvaluator#finishBundle()} */ - private TransformResult finishBundle( + private TransformResult finishBundle( TransformEvaluator evaluator, MetricsContainer metricsContainer, Collection> enforcements) throws Exception { - TransformResult result = evaluator.finishBundle() + TransformResult result = evaluator.finishBundle() .withLogicalMetricUpdates(metricsContainer.getCumulative()); CommittedResult outputs = onComplete.handleResult(inputBundle, result); for (ModelEnforcement enforcement : enforcements) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java index ac1e39568bc1..b4797b01ce15 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.metrics.MetricUpdates; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; @@ -32,16 +33,25 @@ /** * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}. + * + *

Every transform evaluator has a defined input type, but {@link ParDo} has multiple outputs + * so there is not necesssarily a defined output type. */ -public interface TransformResult { +public interface TransformResult { /** * Returns the {@link AppliedPTransform} that produced this result. + * + *

This is treated as an opaque identifier so evaluators can delegate to other evaluators + * that may not have compatible types. */ AppliedPTransform getTransform(); /** * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These * will be committed by the evaluation context as part of completing this result. + * + *

Note that the bundles need not have a uniform type, for example in the case of multi-output + * {@link ParDo}. */ Iterable> getOutputBundles(); @@ -49,7 +59,7 @@ public interface TransformResult { * Returns elements that were provided to the {@link TransformEvaluator} as input but were not * processed. */ - Iterable> getUnprocessedElements(); + Iterable> getUnprocessedElements(); /** * Returns the {@link AggregatorContainer.Mutator} used by this {@link PTransform}, or null if @@ -97,5 +107,5 @@ public interface TransformResult { * Returns a new TransformResult based on this one but overwriting any existing logical metric * updates with {@code metricUpdates}. */ - TransformResult withLogicalMetricUpdates(MetricUpdates metricUpdates); + TransformResult withLogicalMetricUpdates(MetricUpdates metricUpdates); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index 24a91cb3cddc..a4aebc9ee774 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -229,7 +229,8 @@ public void run() { } @Override - public TransformResult finishBundle() throws IOException { + public TransformResult> finishBundle() + throws IOException { return resultBuilder.build(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index 2dd280acabbb..b92ade14655b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -81,7 +81,7 @@ public void processElement(WindowedValue> element) { } @Override - public TransformResult finishBundle() { + public TransformResult> finishBundle() { writer.add(elements); Builder resultBuilder = StepTransformResult.withoutHold(application); if (!elements.isEmpty()) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index eb53b7f29630..991addfb34a7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -103,8 +103,10 @@ private Collection assignWind } @Override - public TransformResult finishBundle() throws Exception { - return StepTransformResult.withoutHold(transform).addOutput(outputBundle).build(); + public TransformResult finishBundle() throws Exception { + return StepTransformResult.withoutHold(transform) + .addOutput(outputBundle) + .build(); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index e956c34d238d..dee95a7819d0 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -110,7 +110,7 @@ public void boundedSourceInMemoryTransformEvaluatorProducesElements() throws Exc for (WindowedValue shard : shardBundle.getElements()) { evaluator.processElement((WindowedValue) shard); } - TransformResult result = evaluator.finishBundle(); + TransformResult result = evaluator.finishBundle(); assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); assertThat( Iterables.size(result.getOutputBundles()), @@ -154,11 +154,11 @@ public void boundedSourceEvaluatorProducesDynamicSplits() throws Exception { Collection> newUnreadInputs = new ArrayList<>(); for (CommittedBundle shardBundle : unreadInputs) { - TransformEvaluator evaluator = factory.forApplication(transform, null); + TransformEvaluator evaluator = factory.forApplication(transform, null); for (WindowedValue shard : shardBundle.getElements()) { evaluator.processElement((WindowedValue) shard); } - TransformResult result = evaluator.finishBundle(); + TransformResult result = evaluator.finishBundle(); assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); assertThat( Iterables.size(result.getOutputBundles()), @@ -207,7 +207,7 @@ public void boundedSourceEvaluatorDynamicSplitsUnsplittable() throws Exception { for (WindowedValue shard : shardBundle.getElements()) { evaluator.processElement((WindowedValue) shard); } - TransformResult result = evaluator.finishBundle(); + TransformResult result = evaluator.finishBundle(); assertThat(result.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); assertThat( Iterables.size(result.getOutputBundles()), @@ -277,7 +277,7 @@ public void boundedSourceInMemoryTransformEvaluatorShardsOfSource() throws Excep when(context.createBundle(longs)).thenReturn(outputBundle); evaluator.processElement(shard); } - TransformResult result = evaluator.finishBundle(); + TransformResult result = evaluator.finishBundle(); assertThat(Iterables.size(result.getOutputBundles()), equalTo(splits.size())); List> outputElems = new ArrayList<>(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 9e2732ee5abe..b5eec63a19cf 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -115,7 +115,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { finishBundleCalled = true; return null; } @@ -128,7 +128,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { throw new Exception(); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index e1277ac5edc4..9a3959dab2c2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -250,7 +250,7 @@ public void handleResultCommitsAggregators() { AggregatorContainer.Mutator mutator = container.createMutator(); mutator.createAggregatorForDoFn(fn, stepContext, "foo", new SumLongFn()).addValue(4L); - TransformResult result = + TransformResult result = StepTransformResult.withoutHold(created.getProducingTransformInternal()) .withAggregatorChanges(mutator) .build(); @@ -260,7 +260,7 @@ public void handleResultCommitsAggregators() { AggregatorContainer.Mutator mutatorAgain = container.createMutator(); mutatorAgain.createAggregatorForDoFn(fn, stepContext, "foo", new SumLongFn()).addValue(12L); - TransformResult secondResult = + TransformResult secondResult = StepTransformResult.withoutHold(downstream.getProducingTransformInternal()) .withAggregatorChanges(mutatorAgain) .build(); @@ -286,7 +286,7 @@ public void handleResultStoresState() { bag.add(2); bag.add(4); - TransformResult stateResult = + TransformResult stateResult = StepTransformResult.withoutHold(downstream.getProducingTransformInternal()) .withState(state) .build(); @@ -319,7 +319,7 @@ public void run() { context.scheduleAfterOutputWouldBeProduced( downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback); - TransformResult result = + TransformResult result = StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0)) .build(); @@ -328,7 +328,7 @@ public void run() { // will likely be flaky if this logic is broken assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false)); - TransformResult finishedResult = + TransformResult finishedResult = StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); context.handleResult(null, ImmutableList.of(), finishedResult); context.forceRefresh(); @@ -338,7 +338,7 @@ public void run() { @Test public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception { - TransformResult finishedResult = + TransformResult finishedResult = StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); context.handleResult(null, ImmutableList.of(), finishedResult); @@ -358,7 +358,7 @@ public void run() { @Test public void extractFiredTimersExtractsTimers() { - TransformResult holdResult = + TransformResult holdResult = StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0)) .build(); context.handleResult(null, ImmutableList.of(), holdResult); @@ -366,7 +366,7 @@ public void extractFiredTimersExtractsTimers() { StructuralKey key = StructuralKey.of("foo".length(), VarIntCoder.of()); TimerData toFire = TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME); - TransformResult timerResult = + TransformResult timerResult = StepTransformResult.withoutHold(downstream.getProducingTransformInternal()) .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null)) .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build()) @@ -382,7 +382,7 @@ public void extractFiredTimersExtractsTimers() { // timer hasn't fired assertThat(context.extractFiredTimers(), emptyIterable()); - TransformResult advanceResult = + TransformResult advanceResult = StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); // Should cause the downstream timer to fire context.handleResult(null, ImmutableList.of(), advanceResult); @@ -460,7 +460,7 @@ public void isDoneWithPartiallyDone() { context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(created.getProducingTransformInternal()) + StepTransformResult.withoutHold(created.getProducingTransformInternal()) .addOutput(rootBundle) .build()); @SuppressWarnings("unchecked") diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 417aa6406ca6..cb27fbc923b9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -84,8 +84,8 @@ public void testFlattenInMemoryEvaluator() throws Exception { rightSideEvaluator.processElement( WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096))); - TransformResult rightSideResult = rightSideEvaluator.finishBundle(); - TransformResult leftSideResult = leftSideEvaluator.finishBundle(); + TransformResult rightSideResult = rightSideEvaluator.finishBundle(); + TransformResult leftSideResult = leftSideEvaluator.finishBundle(); assertThat( rightSideResult.getOutputBundles(), @@ -131,7 +131,7 @@ public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Except flattened.getProducingTransformInternal(), bundleFactory.createRootBundle().commit(BoundedWindow.TIMESTAMP_MAX_VALUE)); - TransformResult leftSideResult = emptyEvaluator.finishBundle(); + TransformResult leftSideResult = emptyEvaluator.finishBundle(); CommittedBundle outputBundle = Iterables.getOnlyElement(leftSideResult.getOutputBundles()).commit(Instant.now()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index a7277fef90a7..a65cd30b0a53 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -78,7 +78,7 @@ public void unchangedSucceeds() { enforcement.afterElement(element); enforcement.afterFinish( elements, - StepTransformResult.withoutHold(consumer).build(), + StepTransformResult.withoutHold(consumer).build(), Collections.>emptyList()); } @@ -98,7 +98,7 @@ public void mutatedDuringProcessElementThrows() { enforcement.afterElement(element); enforcement.afterFinish( elements, - StepTransformResult.withoutHold(consumer).build(), + StepTransformResult.withoutHold(consumer).build(), Collections.>emptyList()); } @@ -120,7 +120,7 @@ public void mutatedAfterProcessElementFails() { thrown.expectMessage("Input values must not be mutated"); enforcement.afterFinish( elements, - StepTransformResult.withoutHold(consumer).build(), + StepTransformResult.withoutHold(consumer).build(), Collections.>emptyList()); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index eab92f41c47a..85e99c51a2b4 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -112,7 +112,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { evaluator.processElement(first); evaluator.processElement(second); evaluator.processElement(third); - TransformResult result = evaluator.finishBundle(); + TransformResult result = evaluator.finishBundle(); assertThat( result.getUnprocessedElements(), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java index 61f581248a8c..a21d8f7c9d0a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java @@ -56,34 +56,37 @@ public void setup() { @Test public void producedBundlesProducedOutputs() { UncommittedBundle bundle = bundleFactory.createBundle(pc); - TransformResult result = StepTransformResult.withoutHold(transform).addOutput(bundle) - .build(); + TransformResult result = + StepTransformResult.withoutHold(transform).addOutput(bundle).build(); - assertThat(result.getOutputBundles(), Matchers.containsInAnyOrder(bundle)); + assertThat( + result.getOutputBundles(), Matchers.>containsInAnyOrder(bundle)); } @Test public void withAdditionalOutputProducedOutputs() { - TransformResult result = StepTransformResult.withoutHold(transform) - .withAdditionalOutput(OutputType.PCOLLECTION_VIEW) - .build(); + TransformResult result = + StepTransformResult.withoutHold(transform) + .withAdditionalOutput(OutputType.PCOLLECTION_VIEW) + .build(); assertThat(result.getOutputTypes(), containsInAnyOrder(OutputType.PCOLLECTION_VIEW)); } @Test public void producedBundlesAndAdditionalOutputProducedOutputs() { - TransformResult result = StepTransformResult.withoutHold(transform) - .addOutput(bundleFactory.createBundle(pc)) - .withAdditionalOutput(OutputType.PCOLLECTION_VIEW) - .build(); + TransformResult result = + StepTransformResult.withoutHold(transform) + .addOutput(bundleFactory.createBundle(pc)) + .withAdditionalOutput(OutputType.PCOLLECTION_VIEW) + .build(); assertThat(result.getOutputTypes(), hasItem(OutputType.PCOLLECTION_VIEW)); } @Test public void noBundlesNoAdditionalOutputProducedOutputsFalse() { - TransformResult result = StepTransformResult.withoutHold(transform).build(); + TransformResult result = StepTransformResult.withoutHold(transform).build(); assertThat(result.getOutputTypes(), emptyIterable()); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index 94a0d41fd3c2..3d31df6d2bb6 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -90,7 +90,7 @@ public void producesElementsInSequence() throws Exception { TransformEvaluator> firstEvaluator = factory.forApplication(streamVals.getProducingTransformInternal(), initialBundle); firstEvaluator.processElement(Iterables.getOnlyElement(initialBundle.getElements())); - TransformResult firstResult = firstEvaluator.finishBundle(); + TransformResult> firstResult = firstEvaluator.finishBundle(); WindowedValue> firstResidual = (WindowedValue>) @@ -103,7 +103,7 @@ public void producesElementsInSequence() throws Exception { TransformEvaluator> secondEvaluator = factory.forApplication(streamVals.getProducingTransformInternal(), secondBundle); secondEvaluator.processElement(firstResidual); - TransformResult secondResult = secondEvaluator.finishBundle(); + TransformResult> secondResult = secondEvaluator.finishBundle(); WindowedValue> secondResidual = (WindowedValue>) @@ -116,7 +116,7 @@ public void producesElementsInSequence() throws Exception { TransformEvaluator> thirdEvaluator = factory.forApplication(streamVals.getProducingTransformInternal(), thirdBundle); thirdEvaluator.processElement(secondResidual); - TransformResult thirdResult = thirdEvaluator.finishBundle(); + TransformResult> thirdResult = thirdEvaluator.finishBundle(); WindowedValue> thirdResidual = (WindowedValue>) @@ -130,7 +130,7 @@ public void producesElementsInSequence() throws Exception { TransformEvaluator> fourthEvaluator = factory.forApplication(streamVals.getProducingTransformInternal(), fourthBundle); fourthEvaluator.processElement(thirdResidual); - TransformResult fourthResult = fourthEvaluator.finishBundle(); + TransformResult> fourthResult = fourthEvaluator.finishBundle(); assertThat(clock.now(), equalTo(start.plus(Duration.standardMinutes(10)))); WindowedValue> fourthResidual = @@ -144,7 +144,7 @@ public void producesElementsInSequence() throws Exception { TransformEvaluator> fifthEvaluator = factory.forApplication(streamVals.getProducingTransformInternal(), fifthBundle); fifthEvaluator.processElement(fourthResidual); - TransformResult fifthResult = fifthEvaluator.finishBundle(); + TransformResult> fifthResult = fifthEvaluator.finishBundle(); assertThat( Iterables.getOnlyElement(firstResult.getOutputBundles()) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java index 0b7b8822abc6..85eff6598e34 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.direct; -import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -96,7 +95,7 @@ public void setup() { @Test public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception { - final TransformResult result = + final TransformResult result = StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); final AtomicBoolean finishCalled = new AtomicBoolean(false); TransformEvaluator evaluator = @@ -107,7 +106,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { finishCalled.set(true); return result; } @@ -128,7 +127,7 @@ public TransformResult finishBundle() throws Exception { executor.run(); assertThat(finishCalled.get(), is(true)); - assertThat(completionCallback.handledResult, equalTo(result)); + assertThat(completionCallback.handledResult, Matchers.>equalTo(result)); assertThat(completionCallback.handledException, is(nullValue())); } @@ -154,8 +153,8 @@ public void nullTransformEvaluatorTerminates() throws Exception { @Test public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception { - final TransformResult result = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); + final TransformResult result = + StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); final Collection> elementsProcessed = new ArrayList<>(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -166,7 +165,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { return result; } }; @@ -194,14 +193,14 @@ public TransformResult finishBundle() throws Exception { evaluatorCompleted.await(); assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo)); - assertThat(completionCallback.handledResult, equalTo(result)); + assertThat(completionCallback.handledResult, Matchers.>equalTo(result)); assertThat(completionCallback.handledException, is(nullValue())); } @Test public void processElementThrowsExceptionCallsback() throws Exception { - final TransformResult result = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); + final TransformResult result = + StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); final Exception exception = new Exception(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -211,7 +210,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { return result; } }; @@ -248,7 +247,7 @@ public void finishBundleThrowsExceptionCallsback() throws Exception { public void processElement(WindowedValue element) throws Exception {} @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { throw exception; } }; @@ -277,7 +276,7 @@ public TransformResult finishBundle() throws Exception { @Test public void callWithEnforcementAppliesEnforcement() throws Exception { - final TransformResult result = + final TransformResult result = StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); TransformEvaluator evaluator = @@ -286,7 +285,7 @@ public void callWithEnforcementAppliesEnforcement() throws Exception { public void processElement(WindowedValue element) throws Exception {} @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { return result; } }; @@ -317,7 +316,7 @@ public TransformResult finishBundle() throws Exception { assertThat( testEnforcement.afterElements, Matchers.>containsInAnyOrder(barElem, fooElem)); - assertThat(testEnforcement.finishedBundles, contains(result)); + assertThat(testEnforcement.finishedBundles, Matchers.>contains(result)); } @Test @@ -333,7 +332,7 @@ public PCollection apply(PCollection input) { } }); - final TransformResult result = + final TransformResult result = StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build(); final CountDownLatch testLatch = new CountDownLatch(1); final CountDownLatch evaluatorLatch = new CountDownLatch(1); @@ -344,7 +343,7 @@ public PCollection apply(PCollection input) { public void processElement(WindowedValue element) throws Exception {} @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { testLatch.countDown(); evaluatorLatch.await(); return result; @@ -389,7 +388,7 @@ public PCollection apply(PCollection input) { } }); - final TransformResult result = + final TransformResult result = StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build(); final CountDownLatch testLatch = new CountDownLatch(1); final CountDownLatch evaluatorLatch = new CountDownLatch(1); @@ -403,7 +402,7 @@ public void processElement(WindowedValue element) throws Exception { } @Override - public TransformResult finishBundle() throws Exception { + public TransformResult finishBundle() throws Exception { return result; } }; @@ -434,7 +433,7 @@ public TransformResult finishBundle() throws Exception { } private static class RegisteringCompletionCallback implements CompletionCallback { - private TransformResult handledResult = null; + private TransformResult handledResult = null; private boolean handledEmpty = false; private Exception handledException = null; private final CountDownLatch onMethod; @@ -444,7 +443,7 @@ private RegisteringCompletionCallback(CountDownLatch onMethod) { } @Override - public CommittedResult handleResult(CommittedBundle inputBundle, TransformResult result) { + public CommittedResult handleResult(CommittedBundle inputBundle, TransformResult result) { handledResult = result; onMethod.countDown(); @SuppressWarnings("rawtypes") @@ -490,7 +489,7 @@ public TestEnforcement forBundle( private static class TestEnforcement implements ModelEnforcement { private final List> beforeElements = new ArrayList<>(); private final List> afterElements = new ArrayList<>(); - private final List finishedBundles = new ArrayList<>(); + private final List> finishedBundles = new ArrayList<>(); @Override public void beforeElement(WindowedValue element) { @@ -505,7 +504,7 @@ public void afterElement(WindowedValue element) { @Override public void afterFinish( CommittedBundle input, - TransformResult result, + TransformResult result, Iterable> outputs) { finishedBundles.add(result); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index 8d382750503f..5a10134fb87c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -159,9 +159,10 @@ public void unboundedSourceInMemoryTransformEvaluatorProducesElements() throws E longs.getProducingTransformInternal(), inputShards); evaluator.processElement((WindowedValue) Iterables.getOnlyElement(inputShards.getElements())); - TransformResult result = evaluator.finishBundle(); + TransformResult> result = evaluator.finishBundle(); - WindowedValue residual = Iterables.getOnlyElement(result.getUnprocessedElements()); + WindowedValue> residual = + Iterables.getOnlyElement(result.getUnprocessedElements()); assertThat( residual.getTimestamp(), Matchers.lessThan(DateTime.now().toInstant())); UnboundedSourceShard residualShard = @@ -206,7 +207,8 @@ public void unboundedSourceWithDuplicatesMultipleCalls() throws Exception { evaluator.processElement( (WindowedValue>) value); } - TransformResult result = evaluator.finishBundle(); + TransformResult> result = + evaluator.finishBundle(); assertThat( output.commit(Instant.now()).getElements(), containsInAnyOrder(tgw(1L), tgw(2L), tgw(4L), tgw(3L), tgw(0L))); @@ -248,7 +250,8 @@ public void noElementsAvailableReaderIncludedInResidual() throws Exception { evaluator.processElement( (WindowedValue>) value); } - TransformResult result = evaluator.finishBundle(); + TransformResult> result = + evaluator.finishBundle(); // Read from the residual of the first read. This should not produce any output, but should // include a residual shard in the result. @@ -261,7 +264,8 @@ public void noElementsAvailableReaderIncludedInResidual() throws Exception { Iterables.getOnlyElement(result.getUnprocessedElements()); secondEvaluator.processElement(residual); - TransformResult secondResult = secondEvaluator.finishBundle(); + TransformResult> secondResult = + secondEvaluator.finishBundle(); // Sanity check that nothing was output (The test would have to run for more than a day to do // so correctly.) @@ -308,7 +312,8 @@ public void evaluatorReusesReader() throws Exception { TransformEvaluator> evaluator = factory.forApplication(sourceTransform, inputBundle); evaluator.processElement(shard); - TransformResult result = evaluator.finishBundle(); + TransformResult> result = + evaluator.finishBundle(); CommittedBundle> residual = inputBundle.withElements( @@ -350,7 +355,8 @@ public void evaluatorClosesReaderAndResumesFromCheckpoint() throws Exception { TransformEvaluator> evaluator = factory.forApplication(sourceTransform, inputBundle); evaluator.processElement(shard); - TransformResult result = evaluator.finishBundle(); + TransformResult> result = + evaluator.finishBundle(); CommittedBundle> residual = inputBundle.withElements( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index 741f8f217236..e2f987c73c13 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -118,7 +118,7 @@ public void nullWindowFunSucceeds() throws Exception { UncommittedBundle outputBundle = createOutputBundle(triggering, inputBundle); - TransformResult result = runEvaluator(triggering, inputBundle, transform); + TransformResult result = runEvaluator(triggering, inputBundle, transform); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -143,7 +143,7 @@ public void singleWindowFnSucceeds() throws Exception { BoundedWindow firstSecondWindow = new IntervalWindow(EPOCH, EPOCH.plus(windowDuration)); BoundedWindow thirdWindow = new IntervalWindow(EPOCH.minus(windowDuration), EPOCH); - TransformResult result = runEvaluator(windowed, inputBundle, transform); + TransformResult result = runEvaluator(windowed, inputBundle, transform); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -178,7 +178,7 @@ public void multipleWindowsWindowFnSucceeds() throws Exception { CommittedBundle inputBundle = createInputBundle(); UncommittedBundle outputBundle = createOutputBundle(windowed, inputBundle); - TransformResult result = runEvaluator(windowed, inputBundle, transform); + TransformResult result = runEvaluator(windowed, inputBundle, transform); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -235,7 +235,7 @@ public void referencesEarlierWindowsSucceeds() throws Exception { CommittedBundle inputBundle = createInputBundle(); UncommittedBundle outputBundle = createOutputBundle(windowed, inputBundle); - TransformResult result = runEvaluator(windowed, inputBundle, transform); + TransformResult result = runEvaluator(windowed, inputBundle, transform); assertThat( Iterables.getOnlyElement(result.getOutputBundles()), @@ -301,7 +301,7 @@ private UncommittedBundle createOutputBundle( return outputBundle; } - private TransformResult runEvaluator( + private TransformResult runEvaluator( PCollection windowed, CommittedBundle inputBundle, Window.Bound windowTransform /* Required while Window.Bound is a composite */) @@ -313,7 +313,7 @@ private TransformResult runEvaluator( evaluator.processElement(valueInGlobalWindow); evaluator.processElement(valueInGlobalAndTwoIntervalWindows); evaluator.processElement(valueInIntervalWindow); - TransformResult result = evaluator.finishBundle(); + TransformResult result = evaluator.finishBundle(); return result; } From 1b7b065f4ccae7c52934b1e73fd1fbfb33c3398d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Nov 2016 11:34:45 -0800 Subject: [PATCH 021/279] Remove overspecified type in ParDoEvaluator --- .../java/org/apache/beam/runners/direct/ParDoEvaluator.java | 6 +++--- .../apache/beam/runners/direct/ParDoEvaluatorFactory.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 254fa4421b5c..3285c7edfc52 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -45,7 +45,7 @@ class ParDoEvaluator implements TransformEvaluator { public static ParDoEvaluator create( EvaluationContext evaluationContext, DirectStepContext stepContext, - AppliedPTransform, ?, ?> application, + AppliedPTransform application, WindowingStrategy windowingStrategy, Serializable fn, // may be OldDoFn or DoFn List> sideInputs, @@ -90,7 +90,7 @@ public static ParDoEvaluator create( //////////////////////////////////////////////////////////////////////////////////////////////// private final PushbackSideInputDoFnRunner fnRunner; - private final AppliedPTransform, ?, ?> transform; + private final AppliedPTransform transform; private final AggregatorContainer.Mutator aggregatorChanges; private final Collection> outputBundles; private final DirectStepContext stepContext; @@ -99,7 +99,7 @@ public static ParDoEvaluator create( private ParDoEvaluator( PushbackSideInputDoFnRunner fnRunner, - AppliedPTransform, ?, ?> transform, + AppliedPTransform transform, AggregatorContainer.Mutator aggregatorChanges, Collection> outputBundles, DirectStepContext stepContext) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index f12600042d4c..b776da1d83fb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -82,7 +82,7 @@ private TransformEvaluator createEvaluator( try { ParDo.BoundMulti transform = application.getTransform(); return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping( - ParDoEvaluator.create( + ParDoEvaluator.create( evaluationContext, stepContext, application, From e17dc4af9f7de717872d6c6f0ab52e0498f3b782 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 9 Nov 2016 21:10:51 -0800 Subject: [PATCH 022/279] Add State parameter support to SimpleDoFnRunner --- .../beam/runners/core/SimpleDoFnRunner.java | 60 ++++++++++++-- .../org/apache/beam/sdk/transforms/ParDo.java | 10 --- .../sdk/transforms/reflect/DoFnSignature.java | 1 + .../apache/beam/sdk/transforms/ParDoTest.java | 79 +++++++++++++++---- 4 files changed, 118 insertions(+), 32 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index f611c0af572f..68751f09a234 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -25,7 +25,9 @@ import java.util.Iterator; import java.util.List; import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunners.OutputManager; +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.Aggregator.AggregatorFactory; @@ -37,6 +39,7 @@ import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -55,6 +58,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.apache.beam.sdk.util.state.StateSpec; +import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Instant; @@ -87,6 +94,13 @@ public class SimpleDoFnRunner implements DoFnRunner windowCoder; + + // Because of setKey(Object), we really must refresh stateInternals() at each access + private final StepContext stepContext; + public SimpleDoFnRunner( PipelineOptions options, DoFn fn, @@ -98,11 +112,20 @@ public SimpleDoFnRunner( AggregatorFactory aggregatorFactory, WindowingStrategy windowingStrategy) { this.fn = fn; - this.observesWindow = - DoFnSignatures.getSignature(fn.getClass()).processElement().observesWindow(); + this.signature = DoFnSignatures.getSignature(fn.getClass()); + this.observesWindow = signature.processElement().observesWindow(); this.invoker = DoFnInvokers.invokerFor(fn); this.outputManager = outputManager; this.mainOutputTag = mainOutputTag; + this.stepContext = stepContext; + + // This is a cast of an _invariant_ coder. But we are assured by pipeline validation + // that it really is the coder for whatever BoundedWindow subclass is provided + @SuppressWarnings("unchecked") + Coder untypedCoder = + (Coder) windowingStrategy.getWindowFn().windowCoder(); + this.windowCoder = untypedCoder; + this.context = new DoFnContext<>( options, @@ -113,7 +136,7 @@ public SimpleDoFnRunner( sideOutputTags, stepContext, aggregatorFactory, - windowingStrategy == null ? null : windowingStrategy.getWindowFn()); + windowingStrategy.getWindowFn()); } @Override @@ -427,6 +450,23 @@ private class DoFnProcessContext extends DoFn. final DoFnContext context; final WindowedValue windowedValue; + /** Lazily initialized; should only be accessed via {@link #getNamespace()}. */ + @Nullable private StateNamespace namespace; + + /** + * The state namespace for this context. + * + *

Any call to {@link #getNamespace()} when more than one window is present will crash; this + * represents a bug in the runner or the {@link DoFnSignature}, since values must be in exactly + * one window when state or timers are relevant. + */ + private StateNamespace getNamespace() { + if (namespace == null) { + namespace = StateNamespaces.window(windowCoder, window()); + } + return namespace; + } + private DoFnProcessContext( DoFn fn, DoFnContext context, @@ -564,8 +604,16 @@ public RestrictionTracker restrictionTracker() { } @Override - public State state(String timerId) { - throw new UnsupportedOperationException("State parameters are not supported."); + public State state(String stateId) { + try { + StateSpec spec = + (StateSpec) signature.stateDeclarations().get(stateId).field().get(fn); + return stepContext + .stateInternals() + .state(getNamespace(), StateTags.tagForSpec(stateId, (StateSpec) spec)); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } } @Override @@ -593,7 +641,7 @@ public TimerInternals timerInternals() { @Override public StateInternals stateInternals() { - return context.stepContext.stateInternals(); + return stepContext.stateInternals(); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 215ae6a675c8..9453294cb832 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -594,16 +594,6 @@ private static void validateWindowTypeForMethod( private static void validate(DoFn fn) { DoFnSignature signature = DoFnSignatures.getSignature((Class) fn.getClass()); - // To be removed when the features are complete and runners have their own adequate - // rejection logic - if (!signature.stateDeclarations().isEmpty()) { - throw new UnsupportedOperationException( - String.format("Found %s annotations on %s, but %s cannot yet be used with state.", - DoFn.StateId.class.getSimpleName(), - fn.getClass().getName(), - DoFn.class.getSimpleName())); - } - // To be removed when the features are complete and runners have their own adequate // rejection logic if (!signature.timerDeclarations().isEmpty()) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index 1c1603099cc9..cd93583f9b86 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -523,6 +523,7 @@ public abstract static class StateDeclaration { static StateDeclaration create( String id, Field field, TypeDescriptor stateType) { + field.setAccessible(true); return new AutoValue_DoFnSignature_StateDeclaration(id, field, stateType); } } 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 3c3e266df155..be1eaa40f62c 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 @@ -36,6 +36,9 @@ import static org.junit.Assert.assertThat; import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -68,6 +71,7 @@ import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; @@ -1459,27 +1463,70 @@ public void populateDisplayData(Builder builder) { assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } - /** - * A test that we properly reject {@link DoFn} implementations that - * include {@link DoFn.StateId} annotations, for now. - */ @Test - public void testUnsupportedState() { - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("cannot yet be used with state"); + @Category(RunnableOnService.class) + public void testValueState() { + final String stateId = "foo"; + + DoFn, Integer> fn = + new DoFn, Integer>() { + + @StateId(stateId) + private final StateSpec> intState = + StateSpecs.value(VarIntCoder.of()); + + @ProcessElement + public void processElement( + ProcessContext c, @StateId(stateId) ValueState state) { + Integer currentValue = MoreObjects.firstNonNull(state.read(), 0); + c.output(currentValue); + state.write(currentValue + 1); + } + }; - DoFn, KV> fn = - new DoFn, KV>() { + Pipeline p = TestPipeline.create(); + PCollection output = + p.apply(Create.of(KV.of("hello", 42), KV.of("hello", 97), KV.of("hello", 84))) + .apply(ParDo.of(fn)); - @StateId("foo") - private final StateSpec> intState = - StateSpecs.value(VarIntCoder.of()); + PAssert.that(output).containsInAnyOrder(0, 1, 2); + p.run(); + } - @ProcessElement - public void processElement(ProcessContext c) { } - }; + @Test + @Category(RunnableOnService.class) + public void testBagSTate() { + final String stateId = "foo"; + + DoFn, List> fn = + new DoFn, List>() { + + @StateId(stateId) + private final StateSpec> bufferState = + StateSpecs.bag(VarIntCoder.of()); + + @ProcessElement + public void processElement( + ProcessContext c, @StateId(stateId) BagState state) { + Iterable currentValue = state.read(); + state.add(c.element().getValue()); + if (Iterables.size(state.read()) >= 4) { + List sorted = Lists.newArrayList(currentValue); + Collections.sort(sorted); + c.output(sorted); + } + } + }; - ParDo.of(fn); + Pipeline p = TestPipeline.create(); + PCollection> output = + p.apply( + Create.of( + KV.of("hello", 97), KV.of("hello", 42), KV.of("hello", 84), KV.of("hello", 12))) + .apply(ParDo.of(fn)); + + PAssert.that(output).containsInAnyOrder(Lists.newArrayList(12, 42, 84, 97)); + p.run(); } @Test From 7e158e4e583372dd79ffaa380ac7c2dbb4846c50 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 21 Nov 2016 15:41:27 -0800 Subject: [PATCH 023/279] Add simple tests for stateful ParDo --- .../apache/beam/sdk/transforms/ParDoTest.java | 106 +++++++++++++++++- 1 file changed, 102 insertions(+), 4 deletions(-) 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 be1eaa40f62c..593f304f9955 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 @@ -55,6 +55,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.UsesStatefulParDo; import org.apache.beam.sdk.transforms.DoFn.OnTimer; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo.Bound; @@ -1464,8 +1465,8 @@ public void populateDisplayData(Builder builder) { } @Test - @Category(RunnableOnService.class) - public void testValueState() { + @Category({RunnableOnService.class, UsesStatefulParDo.class}) + public void testValueStateSimple() { final String stateId = "foo"; DoFn, Integer> fn = @@ -1494,8 +1495,59 @@ public void processElement( } @Test - @Category(RunnableOnService.class) - public void testBagSTate() { + @Category({RunnableOnService.class, UsesStatefulParDo.class}) + public void testValueStateSideOutput() { + final String stateId = "foo"; + + final TupleTag evenTag = new TupleTag() {}; + final TupleTag oddTag = new TupleTag() {}; + + DoFn, Integer> fn = + new DoFn, Integer>() { + + @StateId(stateId) + private final StateSpec> intState = + StateSpecs.value(VarIntCoder.of()); + + @ProcessElement + public void processElement( + ProcessContext c, @StateId(stateId) ValueState state) { + Integer currentValue = MoreObjects.firstNonNull(state.read(), 0); + if (currentValue % 2 == 0) { + c.output(currentValue); + } else { + c.sideOutput(oddTag, currentValue); + } + state.write(currentValue + 1); + } + }; + + Pipeline p = TestPipeline.create(); + PCollectionTuple output = + p.apply( + Create.of( + KV.of("hello", 42), + KV.of("hello", 97), + KV.of("hello", 84), + KV.of("goodbye", 33), + KV.of("hello", 859), + KV.of("goodbye", 83945))) + .apply(ParDo.of(fn).withOutputTags(evenTag, TupleTagList.of(oddTag))); + + PCollection evens = output.get(evenTag); + PCollection odds = output.get(oddTag); + + // There are 0 and 2 from "hello" and just 0 from "goodbye" + PAssert.that(evens).containsInAnyOrder(0, 2, 0); + + // There are 1 and 3 from "hello" and just "1" from "goodbye" + PAssert.that(odds).containsInAnyOrder(1, 3, 1); + p.run(); + } + + @Test + @Category({RunnableOnService.class, UsesStatefulParDo.class}) + public void testBagState() { final String stateId = "foo"; DoFn, List> fn = @@ -1529,6 +1581,52 @@ public void processElement( p.run(); } + @Test + @Category({RunnableOnService.class, UsesStatefulParDo.class}) + public void testBagStateSideInput() { + Pipeline p = TestPipeline.create(); + + final PCollectionView> listView = + p.apply("Create list for side input", Create.of(2, 1, 0)).apply(View.asList()); + + final String stateId = "foo"; + DoFn, List> fn = + new DoFn, List>() { + + @StateId(stateId) + private final StateSpec> bufferState = + StateSpecs.bag(VarIntCoder.of()); + + @ProcessElement + public void processElement( + ProcessContext c, @StateId(stateId) BagState state) { + Iterable currentValue = state.read(); + state.add(c.element().getValue()); + if (Iterables.size(state.read()) >= 4) { + List sorted = Lists.newArrayList(currentValue); + Collections.sort(sorted); + c.output(sorted); + + List sideSorted = Lists.newArrayList(c.sideInput(listView)); + Collections.sort(sideSorted); + c.output(sideSorted); + } + } + }; + + PCollection> output = + p.apply( + "Create main input", + Create.of( + KV.of("hello", 97), KV.of("hello", 42), KV.of("hello", 84), KV.of("hello", 12))) + .apply(ParDo.of(fn).withSideInputs(listView)); + + PAssert.that(output).containsInAnyOrder( + Lists.newArrayList(12, 42, 84, 97), + Lists.newArrayList(0, 1, 2)); + p.run(); + } + @Test public void testWithOutputTagsDisplayData() { DoFn fn = new DoFn() { From ec2c0e0698c1380b309a609eb642aba445c77e27 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 9 Nov 2016 21:59:15 -0800 Subject: [PATCH 024/279] Add support for Stateful ParDo in the Direct runner This adds overrides and new evaluators to ensure that state is accessed in a single-threaded manner per key and is cleaned up when a window expires. --- .../runners/direct/EvaluationContext.java | 15 + .../beam/runners/direct/ParDoEvaluator.java | 11 +- .../runners/direct/ParDoEvaluatorFactory.java | 53 +++- .../direct/ParDoMultiOverrideFactory.java | 76 ++++- .../ParDoSingleViaMultiOverrideFactory.java | 6 +- .../direct/StatefulParDoEvaluatorFactory.java | 256 +++++++++++++++ .../direct/TransformEvaluatorRegistry.java | 2 + .../direct/WatermarkCallbackExecutor.java | 34 ++ .../StatefulParDoEvaluatorFactoryTest.java | 300 ++++++++++++++++++ .../org/apache/beam/sdk/transforms/DoFn.java | 4 +- .../apache/beam/sdk/transforms/OldDoFn.java | 8 +- 11 files changed, 741 insertions(+), 24 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index c1225f63377d..201aaedec526 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -296,6 +296,21 @@ public void scheduleAfterOutputWouldBeProduced( fireAvailableCallbacks(lookupProducing(value)); } + /** + * Schedule a callback to be executed after the given window is expired. + * + *

For example, upstream state associated with the window may be cleared. + */ + public void scheduleAfterWindowExpiration( + AppliedPTransform producing, + BoundedWindow window, + WindowingStrategy windowingStrategy, + Runnable runnable) { + callbackExecutor.callOnWindowExpiration(producing, window, windowingStrategy, runnable); + + fireAvailableCallbacks(producing); + } + private AppliedPTransform getProducing(PValue value) { if (value.getProducingTransformInternal() != null) { return value.getProducingTransformInternal(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 3285c7edfc52..750e5f1d07dc 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -42,6 +42,7 @@ import org.apache.beam.sdk.values.TupleTag; class ParDoEvaluator implements TransformEvaluator { + public static ParDoEvaluator create( EvaluationContext evaluationContext, DirectStepContext stepContext, @@ -84,11 +85,17 @@ public static ParDoEvaluator create( } return new ParDoEvaluator<>( - runner, application, aggregatorChanges, outputBundles.values(), stepContext); + evaluationContext, + runner, + application, + aggregatorChanges, + outputBundles.values(), + stepContext); } //////////////////////////////////////////////////////////////////////////////////////////////// + private final EvaluationContext evaluationContext; private final PushbackSideInputDoFnRunner fnRunner; private final AppliedPTransform transform; private final AggregatorContainer.Mutator aggregatorChanges; @@ -98,11 +105,13 @@ public static ParDoEvaluator create( private final ImmutableList.Builder> unprocessedElements; private ParDoEvaluator( + EvaluationContext evaluationContext, PushbackSideInputDoFnRunner fnRunner, AppliedPTransform transform, AggregatorContainer.Mutator aggregatorChanges, Collection> outputBundles, DirectStepContext stepContext) { + this.evaluationContext = evaluationContext; this.fnRunner = fnRunner; this.transform = transform; this.outputBundles = outputBundles; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index b776da1d83fb..02e034aa74ee 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -20,14 +20,16 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import java.util.List; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.ParDo.BoundMulti; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,10 +56,26 @@ public DoFnLifecycleManager load(DoFn key) throws Exception { @Override public TransformEvaluator forApplication( AppliedPTransform application, CommittedBundle inputBundle) throws Exception { + + AppliedPTransform, PCollectionTuple, ParDo.BoundMulti> + parDoApplication = + (AppliedPTransform< + PCollection, PCollectionTuple, ParDo.BoundMulti>) + application; + + ParDo.BoundMulti transform = parDoApplication.getTransform(); + final DoFn doFn = transform.getNewFn(); + @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator evaluator = (TransformEvaluator) - createEvaluator((AppliedPTransform) application, (CommittedBundle) inputBundle); + createEvaluator( + (AppliedPTransform) application, + inputBundle.getKey(), + doFn, + transform.getSideInputs(), + transform.getMainOutputTag(), + transform.getSideOutputTags().getAll()); return evaluator; } @@ -66,21 +84,32 @@ public void cleanup() throws Exception { DoFnLifecycleManagers.removeAllFromManagers(fnClones.asMap().values()); } + /** + * Creates an evaluator for an arbitrary {@link AppliedPTransform} node, with the pieces of the + * {@link ParDo} unpacked. + * + *

This can thus be invoked regardless of whether the types in the {@link AppliedPTransform} + * correspond with the type in the unpacked {@link DoFn}, side inputs, and output tags. + */ @SuppressWarnings({"unchecked", "rawtypes"}) - private TransformEvaluator createEvaluator( - AppliedPTransform, PCollectionTuple, BoundMulti> - application, - CommittedBundle inputBundle) + TransformEvaluator createEvaluator( + AppliedPTransform, PCollectionTuple, ?> + application, + StructuralKey inputBundleKey, + DoFn doFn, + List> sideInputs, + TupleTag mainOutputTag, + List> sideOutputTags) throws Exception { String stepName = evaluationContext.getStepName(application); DirectStepContext stepContext = evaluationContext - .getExecutionContext(application, inputBundle.getKey()) + .getExecutionContext(application, inputBundleKey) .getOrCreateStepContext(stepName, stepName); - DoFnLifecycleManager fnManager = fnClones.getUnchecked(application.getTransform().getNewFn()); + DoFnLifecycleManager fnManager = fnClones.getUnchecked(doFn); + try { - ParDo.BoundMulti transform = application.getTransform(); return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping( ParDoEvaluator.create( evaluationContext, @@ -88,9 +117,9 @@ private TransformEvaluator createEvaluator( application, application.getInput().getWindowingStrategy(), fnManager.get(), - transform.getSideInputs(), - transform.getMainOutputTag(), - transform.getSideOutputTags().getAll(), + sideInputs, + mainOutputTag, + sideOutputTags, application.getOutput().getAll()), fnManager); } catch (Exception e) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 6cc3e6ee1867..8db51592ba4b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -18,13 +18,19 @@ package org.apache.beam.runners.direct; import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; +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.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypedPValue; /** * A {@link PTransformOverrideFactory} that provides overrides for applications of a {@link ParDo} @@ -42,10 +48,74 @@ public PTransform, PCollectionTuple> override( DoFn fn = transform.getNewFn(); DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); - if (!signature.processElement().isSplittable()) { - return transform; - } else { + if (signature.processElement().isSplittable()) { return new SplittableParDo(fn); + } else if (signature.stateDeclarations().size() > 0 + || signature.timerDeclarations().size() > 0) { + + // Based on the fact that the signature is stateful, DoFnSignatures ensures + // that it is also keyed + ParDo.BoundMulti, OutputT> keyedTransform = + (ParDo.BoundMulti, OutputT>) transform; + + return new GbkThenStatefulParDo(keyedTransform); + } else { + return transform; + } + } + + static class GbkThenStatefulParDo + extends PTransform>, PCollectionTuple> { + private final ParDo.BoundMulti, OutputT> underlyingParDo; + + public GbkThenStatefulParDo(ParDo.BoundMulti, OutputT> underlyingParDo) { + this.underlyingParDo = underlyingParDo; + } + + @Override + public PCollectionTuple apply(PCollection> input) { + + PCollectionTuple outputs = input + .apply("Group by key", GroupByKey.create()) + .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input)); + + return outputs; + } + } + + static class StatefulParDo + extends PTransform>>, PCollectionTuple> { + private final transient ParDo.BoundMulti, OutputT> underlyingParDo; + private final transient PCollection> originalInput; + + public StatefulParDo( + ParDo.BoundMulti, OutputT> underlyingParDo, + PCollection> originalInput) { + this.underlyingParDo = underlyingParDo; + this.originalInput = originalInput; + } + + public ParDo.BoundMulti, OutputT> getUnderlyingParDo() { + return underlyingParDo; + } + + @Override + public Coder getDefaultOutputCoder( + PCollection>> input, TypedPValue output) + throws CannotProvideCoderException { + return underlyingParDo.getDefaultOutputCoder(originalInput, output); + } + + public PCollectionTuple apply(PCollection>> input) { + + PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( + input.getPipeline(), + TupleTagList.of(underlyingParDo.getMainOutputTag()) + .and(underlyingParDo.getSideOutputTags().getAll()), + input.getWindowingStrategy(), + input.isBounded()); + + return outputs; } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java index ee3dfc5c877c..f220a4617f60 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java @@ -54,13 +54,15 @@ public PCollection apply(PCollection input) { // Output tags for ParDo need only be unique up to applied transform TupleTag mainOutputTag = new TupleTag(MAIN_OUTPUT_TAG); - PCollectionTuple output = + PCollectionTuple outputs = input.apply( ParDo.of(underlyingParDo.getNewFn()) .withSideInputs(underlyingParDo.getSideInputs()) .withOutputTags(mainOutputTag, TupleTagList.empty())); + PCollection output = outputs.get(mainOutputTag); - return output.get(mainOutputTag); + output.setTypeDescriptorInternal(underlyingParDo.getNewFn().getOutputTypeDescriptor()); + return output; } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java new file mode 100644 index 000000000000..1f3286c8e7ee --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -0,0 +1,256 @@ +/* + * 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.direct; + +import com.google.auto.value.AutoValue; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.collect.Lists; +import java.util.Collections; +import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.StateDeclaration; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.apache.beam.sdk.util.state.StateSpec; +import org.apache.beam.sdk.util.state.StateTag; +import org.apache.beam.sdk.util.state.StateTags; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; + +/** A {@link TransformEvaluatorFactory} for stateful {@link ParDo}. */ +final class StatefulParDoEvaluatorFactory implements TransformEvaluatorFactory { + + private final LoadingCache, Runnable> + cleanupRegistry; + + private final ParDoEvaluatorFactory, OutputT> delegateFactory; + + StatefulParDoEvaluatorFactory(EvaluationContext evaluationContext) { + this.delegateFactory = new ParDoEvaluatorFactory<>(evaluationContext); + this.cleanupRegistry = + CacheBuilder.newBuilder() + .weakValues() + .build(new CleanupSchedulingLoader(evaluationContext)); + } + + @Override + public TransformEvaluator forApplication( + AppliedPTransform application, CommittedBundle inputBundle) throws Exception { + @SuppressWarnings({"unchecked", "rawtypes"}) + TransformEvaluator evaluator = + (TransformEvaluator) + createEvaluator((AppliedPTransform) application, (CommittedBundle) inputBundle); + return evaluator; + } + + @Override + public void cleanup() throws Exception { + delegateFactory.cleanup(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private TransformEvaluator>> createEvaluator( + AppliedPTransform< + PCollection>>, PCollectionTuple, + StatefulParDo> + application, + CommittedBundle>> inputBundle) + throws Exception { + + final DoFn, OutputT> doFn = + application.getTransform().getUnderlyingParDo().getNewFn(); + final DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + // If the DoFn is stateful, schedule state clearing. + // It is semantically correct to schedule any number of redundant clear tasks; the + // cache is used to limit the number of tasks to avoid performance degradation. + if (signature.stateDeclarations().size() > 0) { + for (final WindowedValue element : inputBundle.getElements()) { + for (final BoundedWindow window : element.getWindows()) { + cleanupRegistry.get( + AppliedPTransformOutputKeyAndWindow.create( + application, (StructuralKey) inputBundle.getKey(), window)); + } + } + } + + TransformEvaluator> delegateEvaluator = + delegateFactory.createEvaluator( + (AppliedPTransform) application, + inputBundle.getKey(), + doFn, + application.getTransform().getUnderlyingParDo().getSideInputs(), + application.getTransform().getUnderlyingParDo().getMainOutputTag(), + application.getTransform().getUnderlyingParDo().getSideOutputTags().getAll()); + + return new StatefulParDoEvaluator<>(delegateEvaluator); + } + + private class CleanupSchedulingLoader + extends CacheLoader, Runnable> { + + private final EvaluationContext evaluationContext; + + public CleanupSchedulingLoader(EvaluationContext evaluationContext) { + this.evaluationContext = evaluationContext; + } + + @Override + public Runnable load( + final AppliedPTransformOutputKeyAndWindow transformOutputWindow) { + String stepName = evaluationContext.getStepName(transformOutputWindow.getTransform()); + + PCollection pc = + transformOutputWindow + .getTransform() + .getOutput() + .get( + transformOutputWindow + .getTransform() + .getTransform() + .getUnderlyingParDo() + .getMainOutputTag()); + WindowingStrategy windowingStrategy = pc.getWindowingStrategy(); + BoundedWindow window = transformOutputWindow.getWindow(); + final DoFn doFn = + transformOutputWindow.getTransform().getTransform().getUnderlyingParDo().getNewFn(); + final DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + final DirectStepContext stepContext = + evaluationContext + .getExecutionContext( + transformOutputWindow.getTransform(), transformOutputWindow.getKey()) + .getOrCreateStepContext(stepName, stepName); + + final StateNamespace namespace = + StateNamespaces.window( + (Coder) windowingStrategy.getWindowFn().windowCoder(), window); + + Runnable cleanup = + new Runnable() { + @Override + public void run() { + for (StateDeclaration stateDecl : signature.stateDeclarations().values()) { + StateTag tag; + try { + tag = + StateTags.tagForSpec(stateDecl.id(), (StateSpec) stateDecl.field().get(doFn)); + } catch (IllegalAccessException e) { + throw new RuntimeException( + String.format( + "Error accessing %s for %s", + StateSpec.class.getName(), doFn.getClass().getName()), + e); + } + stepContext.stateInternals().state(namespace, tag).clear(); + } + cleanupRegistry.invalidate(transformOutputWindow); + } + }; + + evaluationContext.scheduleAfterWindowExpiration( + transformOutputWindow.getTransform(), window, windowingStrategy, cleanup); + return cleanup; + } + } + + @AutoValue + abstract static class AppliedPTransformOutputKeyAndWindow { + abstract AppliedPTransform< + PCollection>>, PCollectionTuple, + StatefulParDo> + getTransform(); + + abstract StructuralKey getKey(); + + abstract BoundedWindow getWindow(); + + static AppliedPTransformOutputKeyAndWindow create( + AppliedPTransform< + PCollection>>, PCollectionTuple, + StatefulParDo> + transform, + StructuralKey key, + BoundedWindow w) { + return new AutoValue_StatefulParDoEvaluatorFactory_AppliedPTransformOutputKeyAndWindow<>( + transform, key, w); + } + } + + private static class StatefulParDoEvaluator + implements TransformEvaluator>> { + + private final TransformEvaluator> delegateEvaluator; + + public StatefulParDoEvaluator(TransformEvaluator> delegateEvaluator) { + this.delegateEvaluator = delegateEvaluator; + } + + @Override + public void processElement(WindowedValue>> gbkResult) throws Exception { + + for (InputT value : gbkResult.getValue().getValue()) { + delegateEvaluator.processElement( + gbkResult.withValue(KV.of(gbkResult.getValue().getKey(), value))); + } + } + + @Override + public TransformResult>> finishBundle() throws Exception { + TransformResult> delegateResult = delegateEvaluator.finishBundle(); + + StepTransformResult.Builder>> regroupedResult = + StepTransformResult.>>withHold( + delegateResult.getTransform(), delegateResult.getWatermarkHold()) + .withTimerUpdate(delegateResult.getTimerUpdate()) + .withAggregatorChanges(delegateResult.getAggregatorChanges()) + .withMetricUpdates(delegateResult.getLogicalMetricUpdates()) + .addOutput(Lists.newArrayList(delegateResult.getOutputBundles())); + + // The delegate may have pushed back unprocessed elements across multiple keys and windows. + // Since processing is single-threaded per key and window, we don't need to regroup the + // outputs, but just make a bunch of singletons + for (WindowedValue untypedUnprocessed : delegateResult.getUnprocessedElements()) { + WindowedValue> windowedKv = (WindowedValue>) untypedUnprocessed; + WindowedValue>> pushedBack = + windowedKv.withValue( + KV.of( + windowedKv.getValue().getKey(), + (Iterable) + Collections.singletonList(windowedKv.getValue().getValue()))); + + regroupedResult.addUnprocessedElements(pushedBack); + } + + return regroupedResult.build(); + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index 0514c3af9d2f..a4c462ab5ec8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -28,6 +28,7 @@ import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList; @@ -50,6 +51,7 @@ public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt) .put(Read.Bounded.class, new BoundedReadEvaluatorFactory(ctxt)) .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory(ctxt)) .put(ParDo.BoundMulti.class, new ParDoEvaluatorFactory<>(ctxt)) + .put(StatefulParDo.class, new StatefulParDoEvaluatorFactory<>(ctxt)) .put(FlattenPCollectionList.class, new FlattenEvaluatorFactory(ctxt)) .put(ViewEvaluatorFactory.WriteView.class, new ViewEvaluatorFactory(ctxt)) .put(Window.Bound.class, new WindowEvaluatorFactory(ctxt)) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java index 54cab7cd5f9f..fcefc5ff1df3 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java @@ -88,6 +88,32 @@ public void callOnGuaranteedFiring( } } + /** + * Execute the provided {@link Runnable} after the next call to + * {@link #fireForWatermark(AppliedPTransform, Instant)} where the window + * is guaranteed to be expired. + */ + public void callOnWindowExpiration( + AppliedPTransform step, + BoundedWindow window, + WindowingStrategy windowingStrategy, + Runnable runnable) { + WatermarkCallback callback = + WatermarkCallback.afterWindowExpiration(window, windowingStrategy, runnable); + + PriorityQueue callbackQueue = callbacks.get(step); + if (callbackQueue == null) { + callbackQueue = new PriorityQueue<>(11, new CallbackOrdering()); + if (callbacks.putIfAbsent(step, callbackQueue) != null) { + callbackQueue = callbacks.get(step); + } + } + + synchronized (callbackQueue) { + callbackQueue.offer(callback); + } + } + /** * Schedule all pending callbacks that must have produced output by the time of the provided * watermark. @@ -112,6 +138,14 @@ public static WatermarkCallback onGuaranteedFiring( return new WatermarkCallback(firingAfter, callback); } + public static WatermarkCallback afterWindowExpiration( + BoundedWindow window, WindowingStrategy strategy, Runnable callback) { + // Fire one milli past the end of the window. This ensures that all window expiration + // timers are delivered first + Instant firingAfter = window.maxTimestamp().plus(strategy.getAllowedLateness()).plus(1L); + return new WatermarkCallback(firingAfter, callback); + } + private final Instant fireAfter; private final Runnable callback; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java new file mode 100644 index 000000000000..ecf11edea3eb --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -0,0 +1,300 @@ +/* + * 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.direct; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.anyList; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; +import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; +import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.apache.beam.sdk.util.state.StateSpec; +import org.apache.beam.sdk.util.state.StateSpecs; +import org.apache.beam.sdk.util.state.StateTag; +import org.apache.beam.sdk.util.state.StateTags; +import org.apache.beam.sdk.util.state.ValueState; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +/** Tests for {@link StatefulParDoEvaluatorFactory}. */ +@RunWith(JUnit4.class) +public class StatefulParDoEvaluatorFactoryTest implements Serializable { + @Mock private transient EvaluationContext mockEvaluationContext; + @Mock private transient DirectExecutionContext mockExecutionContext; + @Mock private transient DirectExecutionContext.DirectStepContext mockStepContext; + @Mock private transient ReadyCheckingSideInputReader mockSideInputReader; + @Mock private transient UncommittedBundle mockUncommittedBundle; + + private static final String KEY = "any-key"; + private transient StateInternals stateInternals = + CopyOnAccessInMemoryStateInternals.withUnderlying(KEY, null); + + private static final BundleFactory BUNDLE_FACTORY = ImmutableListBundleFactory.create(); + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + when((StateInternals) mockStepContext.stateInternals()).thenReturn(stateInternals); + } + + @Test + public void windowCleanupScheduled() throws Exception { + // To test the factory, first we set up a pipeline and then we use the constructed + // pipeline to create the right parameters to pass to the factory + TestPipeline pipeline = TestPipeline.create(); + + final String stateId = "my-state-id"; + + // For consistency, window it into FixedWindows. Actually we will fabricate an input bundle. + PCollection> input = + pipeline + .apply(Create.of(KV.of("hello", 1), KV.of("hello", 2))) + .apply(Window.>into(FixedWindows.of(Duration.millis(10)))); + + PCollection produced = + input.apply( + ParDo.of( + new DoFn, Integer>() { + @StateId(stateId) + private final StateSpec> spec = + StateSpecs.value(StringUtf8Coder.of()); + + @ProcessElement + public void process(ProcessContext c) {} + })); + + StatefulParDoEvaluatorFactory factory = + new StatefulParDoEvaluatorFactory(mockEvaluationContext); + + AppliedPTransform< + PCollection>>, PCollectionTuple, + StatefulParDo> + producingTransform = (AppliedPTransform) produced.getProducingTransformInternal(); + + // Then there will be a digging down to the step context to get the state internals + when(mockEvaluationContext.getExecutionContext( + eq(producingTransform), Mockito.any())) + .thenReturn(mockExecutionContext); + when(mockExecutionContext.getOrCreateStepContext(anyString(), anyString())) + .thenReturn(mockStepContext); + + IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9)); + IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(19)); + + StateNamespace firstWindowNamespace = + StateNamespaces.window(IntervalWindow.getCoder(), firstWindow); + StateNamespace secondWindowNamespace = + StateNamespaces.window(IntervalWindow.getCoder(), secondWindow); + StateTag> tag = + StateTags.tagForSpec(stateId, StateSpecs.value(StringUtf8Coder.of())); + + // Set up non-empty state. We don't mock + verify calls to clear() but instead + // check that state is actually empty. We musn't care how it is accomplished. + stateInternals.state(firstWindowNamespace, tag).write("first"); + stateInternals.state(secondWindowNamespace, tag).write("second"); + + // A single bundle with some elements in the global window; it should register cleanup for the + // global window state merely by having the evaluator created. The cleanup logic does not + // depend on the window. + CommittedBundle> inputBundle = + BUNDLE_FACTORY + .createBundle(input) + .add( + WindowedValue.of( + KV.of("hello", 1), new Instant(3), firstWindow, PaneInfo.NO_FIRING)) + .add( + WindowedValue.of( + KV.of("hello", 2), new Instant(11), secondWindow, PaneInfo.NO_FIRING)) + .commit(Instant.now()); + + // Merely creating the evaluator should suffice to register the cleanup callback + factory.forApplication(producingTransform, inputBundle); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(Runnable.class); + verify(mockEvaluationContext) + .scheduleAfterWindowExpiration( + eq(producingTransform), + eq(firstWindow), + Mockito.>any(), + argumentCaptor.capture()); + + // Should actually clear the state for the first window + argumentCaptor.getValue().run(); + assertThat(stateInternals.state(firstWindowNamespace, tag).read(), nullValue()); + assertThat(stateInternals.state(secondWindowNamespace, tag).read(), equalTo("second")); + + verify(mockEvaluationContext) + .scheduleAfterWindowExpiration( + eq(producingTransform), + eq(secondWindow), + Mockito.>any(), + argumentCaptor.capture()); + + // Should actually clear the state for the second window + argumentCaptor.getValue().run(); + assertThat(stateInternals.state(secondWindowNamespace, tag).read(), nullValue()); + } + + /** + * A test that explicitly delays a side input so that the main input will have to be reprocessed, + * testing that {@code finishBundle()} re-assembles the GBK outputs correctly. + */ + @Test + public void testUnprocessedElements() throws Exception { + // To test the factory, first we set up a pipeline and then we use the constructed + // pipeline to create the right parameters to pass to the factory + TestPipeline pipeline = TestPipeline.create(); + + final String stateId = "my-state-id"; + + // For consistency, window it into FixedWindows. Actually we will fabricate an input bundle. + PCollection> mainInput = + pipeline + .apply(Create.of(KV.of("hello", 1), KV.of("hello", 2))) + .apply(Window.>into(FixedWindows.of(Duration.millis(10)))); + + final PCollectionView> sideInput = + pipeline + .apply("Create side input", Create.of(42)) + .apply("Window side input", Window.into(FixedWindows.of(Duration.millis(10)))) + .apply("View side input", View.asList()); + + PCollection produced = + mainInput.apply( + ParDo.withSideInputs(sideInput) + .of( + new DoFn, Integer>() { + @StateId(stateId) + private final StateSpec> spec = + StateSpecs.value(StringUtf8Coder.of()); + + @ProcessElement + public void process(ProcessContext c) {} + })); + + StatefulParDoEvaluatorFactory factory = + new StatefulParDoEvaluatorFactory(mockEvaluationContext); + + // This will be the stateful ParDo from the expansion + AppliedPTransform< + PCollection>>, PCollectionTuple, + StatefulParDo> + producingTransform = (AppliedPTransform) produced.getProducingTransformInternal(); + + // Then there will be a digging down to the step context to get the state internals + when(mockEvaluationContext.getExecutionContext( + eq(producingTransform), Mockito.any())) + .thenReturn(mockExecutionContext); + when(mockExecutionContext.getOrCreateStepContext(anyString(), anyString())) + .thenReturn(mockStepContext); + when(mockEvaluationContext.createBundle(Matchers.>any())) + .thenReturn(mockUncommittedBundle); + when(mockStepContext.getTimerUpdate()).thenReturn(TimerUpdate.empty()); + + // And digging to check whether the window is ready + when(mockEvaluationContext.createSideInputReader(anyList())).thenReturn(mockSideInputReader); + when(mockSideInputReader.isReady( + Matchers.>any(), Matchers.any())) + .thenReturn(false); + + IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9)); + + // A single bundle with some elements in the global window; it should register cleanup for the + // global window state merely by having the evaluator created. The cleanup logic does not + // depend on the window. + WindowedValue>> gbkOutputElement = + WindowedValue.of( + KV.>of("hello", Lists.newArrayList(1, 13, 15)), + new Instant(3), + firstWindow, + PaneInfo.NO_FIRING); + CommittedBundle>> inputBundle = + BUNDLE_FACTORY + .createBundle(producingTransform.getInput()) + .add(gbkOutputElement) + .commit(Instant.now()); + TransformEvaluator>> evaluator = + factory.forApplication(producingTransform, inputBundle); + evaluator.processElement(gbkOutputElement); + + // This should push back every element as a KV> + // in the appropriate window. Since the keys are equal they are single-threaded + TransformResult>> result = evaluator.finishBundle(); + + List pushedBackInts = new ArrayList<>(); + + for (WindowedValue unprocessedElement : result.getUnprocessedElements()) { + WindowedValue>> unprocessedKv = + (WindowedValue>>) unprocessedElement; + + assertThat( + Iterables.getOnlyElement(unprocessedElement.getWindows()), + equalTo((BoundedWindow) firstWindow)); + assertThat(unprocessedKv.getValue().getKey(), equalTo("hello")); + for (Integer i : unprocessedKv.getValue().getValue()) { + pushedBackInts.add(i); + } + } + assertThat(pushedBackInts, containsInAnyOrder(1, 13, 15)); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 221d942ec4a7..3f1a3f9ad5d0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -315,7 +315,7 @@ public Duration getAllowedTimestampSkew() { * *

See {@link #getOutputTypeDescriptor} for more discussion. */ - protected TypeDescriptor getInputTypeDescriptor() { + public TypeDescriptor getInputTypeDescriptor() { return new TypeDescriptor(getClass()) {}; } @@ -330,7 +330,7 @@ protected TypeDescriptor getInputTypeDescriptor() { * for choosing a default output {@code Coder} for the output * {@code PCollection}. */ - protected TypeDescriptor getOutputTypeDescriptor() { + public TypeDescriptor getOutputTypeDescriptor() { return new TypeDescriptor(getClass()) {}; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index 9bf9003a6af5..2d2c1fde737b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -671,7 +671,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - protected TypeDescriptor getInputTypeDescriptor() { + public TypeDescriptor getInputTypeDescriptor() { return OldDoFn.this.getInputTypeDescriptor(); } @@ -681,7 +681,7 @@ protected TypeDescriptor getInputTypeDescriptor() { } @Override - protected TypeDescriptor getOutputTypeDescriptor() { + public TypeDescriptor getOutputTypeDescriptor() { return OldDoFn.this.getOutputTypeDescriptor(); } } @@ -746,12 +746,12 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - protected TypeDescriptor getInputTypeDescriptor() { + public TypeDescriptor getInputTypeDescriptor() { return OldDoFn.this.getInputTypeDescriptor(); } @Override - protected TypeDescriptor getOutputTypeDescriptor() { + public TypeDescriptor getOutputTypeDescriptor() { return OldDoFn.this.getOutputTypeDescriptor(); } } From 14502a3122bd1c1652196881797ffb60d480164e Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Tue, 22 Nov 2016 08:56:17 -0800 Subject: [PATCH 025/279] Add direct runner dependency to starter archetype --- .../src/main/resources/archetype-resources/pom.xml | 2 +- .../src/main/resources/archetype-resources/pom.xml | 8 ++++++++ .../src/test/resources/projects/basic/reference/pom.xml | 8 ++++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index b18c57c1bd98..031ee88aaf9a 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -93,7 +93,7 @@ 0.4.0-incubating-SNAPSHOT - + org.apache.beam beam-runners-direct-java diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml index 738acde7bd2c..4fae02cf7a02 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml @@ -72,6 +72,14 @@ 0.4.0-incubating-SNAPSHOT + + + org.apache.beam + beam-runners-direct-java + 0.4.0-incubating-SNAPSHOT + runtime + + org.slf4j diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml index 6950ed5f176f..4656e63c1d15 100644 --- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml @@ -72,6 +72,14 @@ 0.4.0-incubating-SNAPSHOT + + + org.apache.beam + beam-runners-direct-java + 0.4.0-incubating-SNAPSHOT + runtime + + org.slf4j From fff577610d929e713d42bfb2a4e8428727ee7206 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 28 Nov 2016 23:33:17 +0100 Subject: [PATCH 026/279] Ignore UsesStatefulParDo Tests in Flink Streaming Runner --- runners/flink/runner/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index c060c254dbf7..18bf6a72a7de 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -79,6 +79,7 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.UsesStatefulParDo none true From fcd09915fac49f66f6a65f234b34dc34f16fef18 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 28 Nov 2016 12:52:14 -0800 Subject: [PATCH 027/279] [BEAM-1009] Upgrade byte-buddy version to 1.5.5. --- pom.xml | 2 +- .../reflect/ByteBuddyDoFnInvokerFactory.java | 23 +++++++++++-------- .../ByteBuddyOnTimerInvokerFactory.java | 10 ++++---- 3 files changed, 19 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index e0ef47c8627b..4e60fabcdd6c 100644 --- a/pom.xml +++ b/pom.xml @@ -754,7 +754,7 @@ net.bytebuddy byte-buddy - 1.4.3 + 1.5.5 diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java index 408ea98e2d51..8750d64b1f75 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java @@ -44,11 +44,11 @@ import net.bytebuddy.implementation.Implementation; import net.bytebuddy.implementation.Implementation.Context; import net.bytebuddy.implementation.MethodDelegation; -import net.bytebuddy.implementation.bind.annotation.TargetMethodAnnotationDrivenBinder; import net.bytebuddy.implementation.bytecode.ByteCodeAppender; import net.bytebuddy.implementation.bytecode.StackManipulation; import net.bytebuddy.implementation.bytecode.Throw; import net.bytebuddy.implementation.bytecode.assign.Assigner; +import net.bytebuddy.implementation.bytecode.assign.Assigner.Typing; import net.bytebuddy.implementation.bytecode.assign.TypeCasting; import net.bytebuddy.implementation.bytecode.constant.TextConstant; import net.bytebuddy.implementation.bytecode.member.FieldAccess; @@ -425,7 +425,7 @@ public Size apply( StackManipulation manipulation = new StackManipulation.Compound( // Push "this" (DoFnInvoker on top of the stack) - MethodVariableAccess.REFERENCE.loadOffset(0), + MethodVariableAccess.REFERENCE.loadFrom(0), // Access this.delegate (DoFn on top of the stack) FieldAccess.forField(delegateField).getter(), // Cast it to the more precise type @@ -471,8 +471,11 @@ protected StackManipulation beforeDelegation(MethodDescription instrumentedMetho * returns void) or contain the value for the instrumented method to return). */ protected StackManipulation afterDelegation(MethodDescription instrumentedMethod) { - return TargetMethodAnnotationDrivenBinder.TerminationHandler.Returning.INSTANCE.resolve( - Assigner.DEFAULT, instrumentedMethod, targetMethod); + return new StackManipulation.Compound( + Assigner.DEFAULT.assign( + targetMethod.getReturnType(), + instrumentedMethod.getReturnType(), Typing.STATIC), + MethodReturn.of(instrumentedMethod.getReturnType())); } } @@ -491,7 +494,7 @@ protected StackManipulation beforeDelegation(MethodDescription instrumentedMetho TypeList.Generic paramTypes = targetMethod.getParameters().asTypeList(); for (int i = 0; i < paramTypes.size(); i++) { TypeDescription.Generic paramT = paramTypes.get(i); - pushParameters.add(MethodVariableAccess.of(paramT).loadOffset(i + 1)); + pushParameters.add(MethodVariableAccess.of(paramT).loadFrom(i + 1)); if (!paramT.isPrimitive()) { pushParameters.add(TypeCasting.to(paramT)); } @@ -634,10 +637,10 @@ protected StackManipulation beforeDelegation(MethodDescription instrumentedMetho // To load the delegate, push `this` and then access the field StackManipulation pushDelegate = new StackManipulation.Compound( - MethodVariableAccess.REFERENCE.loadOffset(0), + MethodVariableAccess.REFERENCE.loadFrom(0), FieldAccess.forField(delegateField).getter()); - StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadOffset(1); + StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadFrom(1); // Push the arguments in their actual order. for (DoFnSignature.Parameter param : signature.extraParameters()) { @@ -654,7 +657,7 @@ protected StackManipulation afterDelegation(MethodDescription instrumentedMethod return new StackManipulation.Compound( MethodInvocation.invoke(PROCESS_CONTINUATION_STOP_METHOD), MethodReturn.REFERENCE); } else { - return MethodReturn.returning(targetMethod.getReturnType().asErasure()); + return MethodReturn.of(targetMethod.getReturnType().asErasure()); } } } @@ -820,9 +823,9 @@ public Size apply( StackManipulation.Size size = new StackManipulation.Compound( // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), + MethodVariableAccess.REFERENCE.loadFrom(0), // Load the delegate argument - MethodVariableAccess.REFERENCE.loadOffset(1), + MethodVariableAccess.REFERENCE.loadFrom(1), // Invoke the super constructor (default constructor of Object) MethodInvocation.invoke( new TypeDescription.ForLoadedType(DoFnInvokerBase.class) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java index 7a39ed104d54..786857ae131c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java @@ -238,10 +238,10 @@ protected StackManipulation beforeDelegation(MethodDescription instrumentedMetho // To load the delegate, push `this` and then access the field StackManipulation pushDelegate = new StackManipulation.Compound( - MethodVariableAccess.REFERENCE.loadOffset(0), + MethodVariableAccess.REFERENCE.loadFrom(0), FieldAccess.forField(delegateField).getter()); - StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadOffset(1); + StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadFrom(1); // Push the extra arguments in their actual order. for (DoFnSignature.Parameter param : signature.extraParameters()) { @@ -275,7 +275,7 @@ public Size apply( StackManipulation.Size size = new StackManipulation.Compound( // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), + MethodVariableAccess.REFERENCE.loadFrom(0), // Invoke the super constructor (default constructor of Object) MethodInvocation.invoke( new TypeDescription.ForLoadedType(Object.class) @@ -285,9 +285,9 @@ public Size apply( .and(ElementMatchers.takesArguments(0))) .getOnly()), // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), + MethodVariableAccess.REFERENCE.loadFrom(0), // Load the delegate argument - MethodVariableAccess.REFERENCE.loadOffset(1), + MethodVariableAccess.REFERENCE.loadFrom(1), // Assign the delegate argument to the delegate field FieldAccess.forField( implementationTarget From 0ae181248823b08803a8f8ac0d39a35a190c4acd Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 28 Nov 2016 15:52:03 -0800 Subject: [PATCH 028/279] Remove the Experimental Annotation from the DirectRunner --- .../java/org/apache/beam/runners/direct/DirectRunner.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index cb319473ff26..f71e109fc5de 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -41,7 +41,6 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.metrics.MetricResults; @@ -71,9 +70,7 @@ * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded * {@link PCollection PCollections}. */ -@Experimental -public class DirectRunner - extends PipelineRunner { +public class DirectRunner extends PipelineRunner { /** * The default set of transform overrides to use in the {@link DirectRunner}. * From ee52318f2512c6661823e4f546f84dbc2caa955b Mon Sep 17 00:00:00 2001 From: sammcveety Date: Fri, 21 Oct 2016 12:50:01 -0400 Subject: [PATCH 029/279] Add method to output runtime options --- .../beam/sdk/options/PipelineOptions.java | 7 +++++ .../sdk/options/PipelineOptionsFactory.java | 1 + .../sdk/options/ProxyInvocationHandler.java | 26 +++++++++++++++++++ .../beam/sdk/options/PipelineOptionsTest.java | 24 +++++++++++++++++ 4 files changed, 58 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java index 2139ed95d9d0..ddb040de168d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java @@ -24,6 +24,7 @@ import com.google.auto.service.AutoService; import com.google.common.base.MoreObjects; import java.lang.reflect.Proxy; +import java.util.Map; import java.util.ServiceLoader; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; @@ -321,6 +322,12 @@ public String create(PipelineOptions options) { } } + /** + * Returns a map of properties which correspond to {@link ValueProvider.RuntimeValueProvider}, + * keyed by the property name. The value is a map containing type and default information. + */ + Map> outputRuntimeOptions(); + /** * Provides a unique ID for this {@link PipelineOptions} object, assigned at graph * construction time. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index 600986761173..9805489c8c28 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -1219,6 +1219,7 @@ private static void validateMethodsAreEitherBeanMethodOrKnownMethod( // Ignore methods on the base PipelineOptions interface. try { knownMethods.add(iface.getMethod("as", Class.class)); + knownMethods.add(iface.getMethod("outputRuntimeOptions")); knownMethods.add(iface.getMethod("populateDisplayData", DisplayData.Builder.class)); } catch (NoSuchMethodException | SecurityException e) { throw new RuntimeException(e); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java index 86f9918389e2..a0e3ec207f98 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java @@ -48,6 +48,7 @@ import java.lang.annotation.Annotation; import java.lang.reflect.InvocationHandler; import java.lang.reflect.Method; +import java.lang.reflect.ParameterizedType; import java.lang.reflect.Proxy; import java.lang.reflect.Type; import java.util.Arrays; @@ -130,6 +131,8 @@ public Object invoke(Object proxy, Method method, Object[] args) { return equals(args[0]); } else if (args == null && "hashCode".equals(method.getName())) { return hashCode(); + } else if (args == null && "outputRuntimeOptions".equals(method.getName())) { + return outputRuntimeOptions((PipelineOptions) proxy); } else if (args != null && "as".equals(method.getName()) && args[0] instanceof Class) { @SuppressWarnings("unchecked") Class clazz = (Class) args[0]; @@ -241,6 +244,29 @@ public int hashCode() { return hashCode; } + /** + * Returns a map of properties which correspond to {@link RuntimeValueProvider}. + */ + public Map> outputRuntimeOptions(PipelineOptions options) { + Set optionSpecs = PipelineOptionsReflector.getOptionSpecs(knownInterfaces); + Map> properties = Maps.newHashMap(); + + for (PipelineOptionSpec spec : optionSpecs) { + if (spec.getGetterMethod().getReturnType().equals(ValueProvider.class)) { + Object vp = invoke(options, spec.getGetterMethod(), null); + if (((ValueProvider) vp).isAccessible()) { + continue; + } + Map property = Maps.newHashMap(); + property.put("type", + ((ParameterizedType) spec.getGetterMethod() + .getGenericReturnType()).getActualTypeArguments()[0]); + properties.put(spec.getName(), property); + } + } + return properties; + } + /** * Nested class to handle display data in order to set the display data namespace to something * sensible. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java index 70a6c986a38e..f7ff45d54410 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java @@ -17,10 +17,13 @@ */ package org.apache.beam.sdk.options; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.ImmutableMap; import java.util.List; +import java.util.Map; import java.util.Set; import org.junit.Rule; import org.junit.Test; @@ -70,4 +73,25 @@ public void testDynamicAs() { BaseTestOptions options = PipelineOptionsFactory.create().as(BaseTestOptions.class); assertNotNull(options); } + + private interface ValueProviderOptions extends PipelineOptions { + ValueProvider getBool(); + void setBool(ValueProvider value); + + ValueProvider getString(); + void setString(ValueProvider value); + + String getNotAValueProvider(); + void setNotAValueProvider(String value); + } + + @Test + public void testOutputRuntimeOptions() { + ValueProviderOptions options = + PipelineOptionsFactory.fromArgs( + new String[]{"--string=baz"}).as(ValueProviderOptions.class); + Map expected = ImmutableMap.of( + "bool", ImmutableMap.of("type", Boolean.class)); + assertEquals(expected, options.outputRuntimeOptions()); + } } From f92259814964fb4d3b2381187247b3f11b5fe33f Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sat, 29 Oct 2016 19:02:51 -0700 Subject: [PATCH 030/279] Demonstrate PubsubIO with NVP --- .../java/org/apache/beam/sdk/io/PubsubIO.java | 176 +++++++++++++++--- .../beam/sdk/io/PubsubUnboundedSink.java | 23 ++- .../beam/sdk/io/PubsubUnboundedSource.java | 40 ++-- .../org/apache/beam/sdk/io/PubsubIOTest.java | 43 +++-- .../beam/sdk/io/PubsubUnboundedSinkTest.java | 20 +- .../sdk/io/PubsubUnboundedSourceTest.java | 14 +- 6 files changed, 232 insertions(+), 84 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index 72a6399b57cd..9768788ef7a5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -31,11 +31,15 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.util.CoderUtils; @@ -134,7 +138,7 @@ private static void validatePubsubName(String name) { * Populate common {@link DisplayData} between Pubsub source and sink. */ private static void populateCommonDisplayData(DisplayData.Builder builder, - String timestampLabel, String idLabel, PubsubTopic topic) { + String timestampLabel, String idLabel, String topic) { builder .addIfNotNull(DisplayData.item("timestampLabel", timestampLabel) .withLabel("Timestamp Label Attribute")) @@ -142,7 +146,7 @@ private static void populateCommonDisplayData(DisplayData.Builder builder, .withLabel("ID Label Attribute")); if (topic != null) { - builder.add(DisplayData.item("topic", topic.asPath()) + builder.add(DisplayData.item("topic", topic) .withLabel("Pubsub Topic")); } } @@ -252,6 +256,61 @@ public String asPath() { } } + /** + * Used to build a {@link ValueProvider} for {@link PubsubSubscription}. + */ + private static class SubscriptionTranslator + implements SerializableFunction { + @Override + public PubsubSubscription apply(String from) { + return PubsubSubscription.fromPath(from); + } + } + + /** + * Used to build a {@link ValueProvider} for {@link SubscriptionPath}. + */ + private static class SubscriptionPathTranslator + implements SerializableFunction { + @Override + public SubscriptionPath apply(PubsubSubscription from) { + return PubsubClient.subscriptionPathFromName(from.project, from.subscription); + } + } + + /** + * Used to build a {@link ValueProvider} for {@link PubsubTopic}. + */ + private static class TopicTranslator + implements SerializableFunction { + @Override + public PubsubTopic apply(String from) { + return PubsubTopic.fromPath(from); + } + } + + /** + * Used to build a {@link ValueProvider} for {@link TopicPath}. + */ + private static class TopicPathTranslator + implements SerializableFunction { + @Override + public TopicPath apply(PubsubTopic from) { + return PubsubClient.topicPathFromName(from.project, from.topic); + } + } + + /** + * Used to build a {@link ValueProvider} for {@link ProjectPath}. + */ + private static class ProjectPathTranslator + implements SerializableFunction { + @Override + public ProjectPath apply(PubsubTopic from) { + return PubsubClient.projectPathFromId(from.project); + } + } + /** * Class representing a Cloud Pub/Sub Topic. */ @@ -380,6 +439,13 @@ public static class Read { * by the runner. */ public static Bound topic(String topic) { + return new Bound<>(DEFAULT_PUBSUB_CODER).topic(StaticValueProvider.of(topic)); + } + + /** + * Like {@code topic()} but with a {@link ValueProvider}. + */ + public static Bound topic(ValueProvider topic) { return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic); } @@ -391,6 +457,13 @@ public static Bound topic(String topic) { * of the {@code subscription} string. */ public static Bound subscription(String subscription) { + return new Bound<>(DEFAULT_PUBSUB_CODER).subscription(StaticValueProvider.of(subscription)); + } + + /** + * Like {@code topic()} but with a {@link ValueProvider}. + */ + public static Bound subscription(ValueProvider subscription) { return new Bound<>(DEFAULT_PUBSUB_CODER).subscription(subscription); } @@ -484,10 +557,10 @@ public static Bound maxReadTime(Duration maxReadTime) { */ public static class Bound extends PTransform> { /** The Cloud Pub/Sub topic to read from. */ - @Nullable private final PubsubTopic topic; + @Nullable private final ValueProvider topic; /** The Cloud Pub/Sub subscription to read from. */ - @Nullable private final PubsubSubscription subscription; + @Nullable private final ValueProvider subscription; /** The name of the message attribute to read timestamps from. */ @Nullable private final String timestampLabel; @@ -508,9 +581,9 @@ private Bound(Coder coder) { this(null, null, null, null, coder, null, 0, null); } - private Bound(String name, PubsubSubscription subscription, PubsubTopic topic, - String timestampLabel, Coder coder, String idLabel, int maxNumRecords, - Duration maxReadTime) { + private Bound(String name, ValueProvider subscription, + ValueProvider topic, String timestampLabel, Coder coder, + String idLabel, int maxNumRecords, Duration maxReadTime) { super(name); this.subscription = subscription; this.topic = topic; @@ -535,8 +608,16 @@ private Bound(String name, PubsubSubscription subscription, PubsubTopic topic, *

Does not modify this object. */ public Bound subscription(String subscription) { - return new Bound<>(name, PubsubSubscription.fromPath(subscription), topic, timestampLabel, - coder, idLabel, maxNumRecords, maxReadTime); + return subscription(StaticValueProvider.of(subscription)); + } + + /** + * Like {@code subscription()} but with a {@link ValueProvider}. + */ + public Bound subscription(ValueProvider subscription) { + return new Bound<>(name, + NestedValueProvider.of(subscription, new SubscriptionTranslator()), + topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** @@ -548,8 +629,16 @@ public Bound subscription(String subscription) { *

Does not modify this object. */ public Bound topic(String topic) { - return new Bound<>(name, subscription, PubsubTopic.fromPath(topic), timestampLabel, coder, - idLabel, maxNumRecords, maxReadTime); + return topic(StaticValueProvider.of(topic)); + } + + /** + * Like {@code topic()} but with a {@link ValueProvider}. + */ + public Bound topic(ValueProvider topic) { + return new Bound<>(name, subscription, + NestedValueProvider.of(topic, new TopicTranslator()), + timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** @@ -629,15 +718,14 @@ public PCollection apply(PBegin input) { .apply(ParDo.of(new PubsubBoundedReader())) .setCoder(coder); } else { - @Nullable ProjectPath projectPath = - topic == null ? null : PubsubClient.projectPathFromId(topic.project); - @Nullable TopicPath topicPath = - topic == null ? null : PubsubClient.topicPathFromName(topic.project, topic.topic); - @Nullable SubscriptionPath subscriptionPath = + @Nullable ValueProvider projectPath = + topic == null ? null : NestedValueProvider.of(topic, new ProjectPathTranslator()); + @Nullable ValueProvider topicPath = + topic == null ? null : NestedValueProvider.of(topic, new TopicPathTranslator()); + @Nullable ValueProvider subscriptionPath = subscription == null ? null - : PubsubClient.subscriptionPathFromName( - subscription.project, subscription.subscription); + : NestedValueProvider.of(subscription, new SubscriptionPathTranslator()); return input.getPipeline().begin() .apply(new PubsubUnboundedSource( FACTORY, projectPath, topicPath, subscriptionPath, @@ -648,7 +736,11 @@ public PCollection apply(PBegin input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - populateCommonDisplayData(builder, timestampLabel, idLabel, topic); + String topicString = + topic == null ? null + : topic.isAccessible() ? topic.get().asPath() + : topic.toString(); + populateCommonDisplayData(builder, timestampLabel, idLabel, topicString); builder .addIfNotNull(DisplayData.item("maxReadTime", maxReadTime) @@ -657,8 +749,10 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Maximum Read Records"), 0); if (subscription != null) { - builder.add(DisplayData.item("subscription", subscription.asPath()) - .withLabel("Pubsub Subscription")); + String subscriptionString = subscription.isAccessible() + ? subscription.get().asPath() : subscription.toString(); + builder.add(DisplayData.item("subscription", subscriptionString) + .withLabel("Pubsub Subscription")); } } @@ -668,10 +762,18 @@ protected Coder getDefaultOutputCoder() { } public PubsubTopic getTopic() { + return topic == null ? null : topic.get(); + } + + public ValueProvider getTopicProvider() { return topic; } public PubsubSubscription getSubscription() { + return subscription == null ? null : subscription.get(); + } + + public ValueProvider getSubscriptionProvider() { return subscription; } @@ -820,6 +922,13 @@ public static class Write { * {@code topic} string. */ public static Bound topic(String topic) { + return new Bound<>(DEFAULT_PUBSUB_CODER).topic(StaticValueProvider.of(topic)); + } + + /** + * Like {@code topic()} but with a {@link ValueProvider}. + */ + public static Bound topic(ValueProvider topic) { return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic); } @@ -869,7 +978,7 @@ public static Bound withCoder(Coder coder) { */ public static class Bound extends PTransform, PDone> { /** The Cloud Pub/Sub topic to publish to. */ - @Nullable private final PubsubTopic topic; + @Nullable private final ValueProvider topic; /** The name of the message attribute to publish message timestamps in. */ @Nullable private final String timestampLabel; /** The name of the message attribute to publish unique message IDs in. */ @@ -881,7 +990,8 @@ private Bound(Coder coder) { } private Bound( - String name, PubsubTopic topic, String timestampLabel, String idLabel, Coder coder) { + String name, ValueProvider topic, String timestampLabel, + String idLabel, Coder coder) { super(name); this.topic = topic; this.timestampLabel = timestampLabel; @@ -899,7 +1009,15 @@ private Bound( *

Does not modify this object. */ public Bound topic(String topic) { - return new Bound<>(name, PubsubTopic.fromPath(topic), timestampLabel, idLabel, coder); + return topic(StaticValueProvider.of(topic)); + } + + /** + * Like {@code topic()} but with a {@link ValueProvider}. + */ + public Bound topic(ValueProvider topic) { + return new Bound<>(name, NestedValueProvider.of(topic, new TopicTranslator()), + timestampLabel, idLabel, coder); } /** @@ -950,7 +1068,7 @@ public PDone apply(PCollection input) { case UNBOUNDED: return input.apply(new PubsubUnboundedSink( FACTORY, - PubsubClient.topicPathFromName(topic.project, topic.topic), + NestedValueProvider.of(topic, new TopicPathTranslator()), coder, timestampLabel, idLabel, @@ -962,7 +1080,9 @@ public PDone apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - populateCommonDisplayData(builder, timestampLabel, idLabel, topic); + String topicString = topic.isAccessible() + ? topic.get().asPath() : topic.toString(); + populateCommonDisplayData(builder, timestampLabel, idLabel, topicString); } @Override @@ -971,6 +1091,10 @@ protected Coder getDefaultOutputCoder() { } public PubsubTopic getTopic() { + return topic.get(); + } + + public ValueProvider getTopicProvider() { return topic; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java index 179abf6da4f6..1e369c8393af 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; @@ -205,7 +206,7 @@ public void populateDisplayData(Builder builder) { private static class WriterFn extends DoFn>, Void> { private final PubsubClientFactory pubsubFactory; - private final TopicPath topic; + private final ValueProvider topic; private final String timestampLabel; private final String idLabel; private final int publishBatchSize; @@ -225,8 +226,8 @@ private static class WriterFn createAggregator("bytes", new Sum.SumLongFn()); WriterFn( - PubsubClientFactory pubsubFactory, TopicPath topic, String timestampLabel, - String idLabel, int publishBatchSize, int publishBatchBytes) { + PubsubClientFactory pubsubFactory, ValueProvider topic, + String timestampLabel, String idLabel, int publishBatchSize, int publishBatchBytes) { this.pubsubFactory = pubsubFactory; this.topic = topic; this.timestampLabel = timestampLabel; @@ -241,7 +242,7 @@ private static class WriterFn */ private void publishBatch(List messages, int bytes) throws IOException { - int n = pubsubClient.publish(topic, messages); + int n = pubsubClient.publish(topic.get(), messages); checkState(n == messages.size(), "Attempted to publish %s messages but %s were successful", messages.size(), n); batchCounter.addValue(1L); @@ -290,7 +291,11 @@ public void finishBundle(Context c) throws Exception { @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("topic", topic.getPath())); + String topicString = + topic == null ? null + : topic.isAccessible() ? topic.get().getPath() + : topic.toString(); + builder.add(DisplayData.item("topic", topicString)); builder.add(DisplayData.item("transport", pubsubFactory.getKind())); builder.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel)); builder.addIfNotNull(DisplayData.item("idLabel", idLabel)); @@ -309,7 +314,7 @@ public void populateDisplayData(Builder builder) { /** * Pubsub topic to publish to. */ - private final TopicPath topic; + private final ValueProvider topic; /** * Coder for elements. It is the responsibility of the underlying Pubsub transport to @@ -363,7 +368,7 @@ public void populateDisplayData(Builder builder) { @VisibleForTesting PubsubUnboundedSink( PubsubClientFactory pubsubFactory, - TopicPath topic, + ValueProvider topic, Coder elementCoder, String timestampLabel, String idLabel, @@ -386,7 +391,7 @@ public void populateDisplayData(Builder builder) { public PubsubUnboundedSink( PubsubClientFactory pubsubFactory, - TopicPath topic, + ValueProvider topic, Coder elementCoder, String timestampLabel, String idLabel, @@ -397,7 +402,7 @@ public PubsubUnboundedSink( } public TopicPath getTopic() { - return topic; + return topic.get(); } @Nullable diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java index bfacb716a144..4ec838924e64 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java @@ -52,6 +52,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; @@ -1161,7 +1162,7 @@ private static class StatsFn extends DoFn { createAggregator("elements", new Sum.SumLongFn()); private final PubsubClientFactory pubsubFactory; - private final SubscriptionPath subscription; + private final ValueProvider subscription; @Nullable private final String timestampLabel; @Nullable @@ -1169,7 +1170,7 @@ private static class StatsFn extends DoFn { public StatsFn( PubsubClientFactory pubsubFactory, - SubscriptionPath subscription, + ValueProvider subscription, @Nullable String timestampLabel, @Nullable @@ -1189,7 +1190,11 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("subscription", subscription.getPath())); + String subscriptionString = + subscription == null ? null + : subscription.isAccessible() ? subscription.get().getPath() + : subscription.toString(); + builder.add(DisplayData.item("subscription", subscriptionString)); builder.add(DisplayData.item("transport", pubsubFactory.getKind())); builder.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel)); builder.addIfNotNull(DisplayData.item("idLabel", idLabel)); @@ -1215,14 +1220,14 @@ public void populateDisplayData(Builder builder) { * Project under which to create a subscription if only the {@link #topic} was given. */ @Nullable - private final ProjectPath project; + private final ValueProvider project; /** * Topic to read from. If {@literal null}, then {@link #subscription} must be given. * Otherwise {@link #subscription} must be null. */ @Nullable - private final TopicPath topic; + private final ValueProvider topic; /** * Subscription to read from. If {@literal null} then {@link #topic} must be given. @@ -1233,7 +1238,7 @@ public void populateDisplayData(Builder builder) { * subscription is never deleted. */ @Nullable - private SubscriptionPath subscription; + private ValueProvider subscription; /** * Coder for elements. Elements are effectively double-encoded: first to a byte array @@ -1260,9 +1265,9 @@ public void populateDisplayData(Builder builder) { PubsubUnboundedSource( Clock clock, PubsubClientFactory pubsubFactory, - @Nullable ProjectPath project, - @Nullable TopicPath topic, - @Nullable SubscriptionPath subscription, + @Nullable ValueProvider project, + @Nullable ValueProvider topic, + @Nullable ValueProvider subscription, Coder elementCoder, @Nullable String timestampLabel, @Nullable String idLabel) { @@ -1285,9 +1290,9 @@ public void populateDisplayData(Builder builder) { */ public PubsubUnboundedSource( PubsubClientFactory pubsubFactory, - @Nullable ProjectPath project, - @Nullable TopicPath topic, - @Nullable SubscriptionPath subscription, + @Nullable ValueProvider project, + @Nullable ValueProvider topic, + @Nullable ValueProvider subscription, Coder elementCoder, @Nullable String timestampLabel, @Nullable String idLabel) { @@ -1300,17 +1305,17 @@ public Coder getElementCoder() { @Nullable public ProjectPath getProject() { - return project; + return project == null ? null : project.get(); } @Nullable public TopicPath getTopic() { - return topic; + return topic == null ? null : topic.get(); } @Nullable public SubscriptionPath getSubscription() { - return subscription; + return subscription == null ? null : subscription.get(); } @Nullable @@ -1335,8 +1340,11 @@ private SubscriptionPath createRandomSubscription(PipelineOptions options) { try { try (PubsubClient pubsubClient = pubsubFactory.newClient(timestampLabel, idLabel, options.as(PubsubOptions.class))) { + checkState(project.isAccessible(), "createRandomSubscription must be called at runtime."); + checkState(topic.isAccessible(), "createRandomSubscription must be called at runtime."); SubscriptionPath subscriptionPath = - pubsubClient.createRandomSubscription(project, topic, DEAULT_ACK_TIMEOUT_SEC); + pubsubClient.createRandomSubscription( + project.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC); LOG.warn( "Created subscription {} to topic {}." + " Note this subscription WILL NOT be deleted when the pipeline terminates", diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java index 086b72644581..b73afb2df894 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java @@ -20,9 +20,12 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import java.util.Set; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; @@ -64,30 +67,14 @@ public void testTopicValidationSuccess() throws Exception { .toString()); } - @Test - public void testTopicValidationBadCharacter() throws Exception { - thrown.expect(IllegalArgumentException.class); - PubsubIO.Read.topic("projects/my-project/topics/abc-*-abc"); - } - - @Test - public void testTopicValidationTooLong() throws Exception { - thrown.expect(IllegalArgumentException.class); - PubsubIO.Read.topic(new StringBuilder().append("projects/my-project/topics/A-really-long-one-") - .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111") - .append("111111111111111111111111111111111111111111111111111111111111111111111111111111111") - .append("1111111111111111111111111111111111111111111111111111111111111111111111111111") - .toString()); - } - @Test public void testReadDisplayData() { String topic = "projects/project/topics/topic"; String subscription = "projects/project/subscriptions/subscription"; Duration maxReadTime = Duration.standardMinutes(5); PubsubIO.Read.Bound read = PubsubIO.Read - .topic(topic) - .subscription(subscription) + .topic(StaticValueProvider.of(topic)) + .subscription(StaticValueProvider.of(subscription)) .timestampLabel("myTimestamp") .idLabel("myId") .maxNumRecords(1234) @@ -103,6 +90,26 @@ public void testReadDisplayData() { assertThat(displayData, hasDisplayItem("maxReadTime", maxReadTime)); } + @Test + public void testNullTopic() { + String subscription = "projects/project/subscriptions/subscription"; + PubsubIO.Read.Bound read = PubsubIO.Read + .subscription(StaticValueProvider.of(subscription)); + assertNull(read.getTopic()); + assertNotNull(read.getSubscription()); + assertNotNull(DisplayData.from(read)); + } + + @Test + public void testNullSubscription() { + String topic = "projects/project/topics/topic"; + PubsubIO.Read.Bound read = PubsubIO.Read + .topic(StaticValueProvider.of(topic)); + assertNotNull(read.getTopic()); + assertNull(read.getSubscription()); + assertNotNull(DisplayData.from(read)); + } + @Test @Category(RunnableOnService.class) public void testPrimitiveReadDisplayData() { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java index 4edd9c106ea3..518136fa1d7b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java @@ -25,6 +25,7 @@ import java.util.List; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.PubsubUnboundedSink.RecordIdMethod; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; @@ -84,9 +85,9 @@ public void sendOneMessage() throws IOException { PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = - new PubsubUnboundedSink<>(factory, TOPIC, StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, - NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), - RecordIdMethod.DETERMINISTIC); + new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(), + TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes, + Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); TestPipeline p = TestPipeline.create(); p.apply(Create.of(ImmutableList.of(DATA))) .apply(ParDo.of(new Stamp())) @@ -113,9 +114,9 @@ public void sendMoreThanOneBatchByNumMessages() throws IOException { PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = - new PubsubUnboundedSink<>(factory, TOPIC, StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, - NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), - RecordIdMethod.DETERMINISTIC); + new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(), + TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes, + Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); TestPipeline p = TestPipeline.create(); p.apply(Create.of(data)) .apply(ParDo.of(new Stamp())) @@ -148,9 +149,10 @@ public void sendMoreThanOneBatchByByteSize() throws IOException { PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = - new PubsubUnboundedSink<>(factory, TOPIC, StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, - NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), - RecordIdMethod.DETERMINISTIC); + new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), + StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, + NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), + RecordIdMethod.DETERMINISTIC); TestPipeline p = TestPipeline.create(); p.apply(Create.of(data)) .apply(ParDo.of(new Stamp())) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java index bbc6c12af299..f6165c507526 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java @@ -45,6 +45,7 @@ import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.util.CoderUtils; @@ -91,8 +92,9 @@ public long currentTimeMillis() { }; factory = PubsubTestClient.createFactoryForPull(clock, SUBSCRIPTION, ACK_TIMEOUT_S, incoming); PubsubUnboundedSource source = - new PubsubUnboundedSource<>(clock, factory, null, null, SUBSCRIPTION, StringUtf8Coder.of(), - TIMESTAMP_LABEL, ID_LABEL); + new PubsubUnboundedSource<>( + clock, factory, null, null, StaticValueProvider.of(SUBSCRIPTION), + StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL); primSource = new PubsubSource<>(source); } @@ -332,8 +334,8 @@ public void noSubscriptionSplitIntoBundlesGeneratesSubscription() throws Excepti PubsubUnboundedSource source = new PubsubUnboundedSource<>( factory, - PubsubClient.projectPathFromId("my_project"), - topicPath, + StaticValueProvider.of(PubsubClient.projectPathFromId("my_project")), + StaticValueProvider.of(topicPath), null, StringUtf8Coder.of(), null, @@ -363,8 +365,8 @@ public void noSubscriptionNoSplitGeneratesSubscription() throws Exception { PubsubUnboundedSource source = new PubsubUnboundedSource<>( factory, - PubsubClient.projectPathFromId("my_project"), - topicPath, + StaticValueProvider.of(PubsubClient.projectPathFromId("my_project")), + StaticValueProvider.of(topicPath), null, StringUtf8Coder.of(), null, From b453457f5699963f9a72129ab86ef18b71c04e61 Mon Sep 17 00:00:00 2001 From: DavidB Date: Fri, 18 Nov 2016 15:11:47 +0000 Subject: [PATCH 031/279] [BEAM-1006] Fixing the MongoDbIO splitKeysToFilters, calling to string on the BSON document prefixed Document to the string, updated it to just get the value of ID --- .../main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index cd5cdafa5572..70239e672a8e 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -302,12 +302,12 @@ private static List splitKeysToFilters(List splitKeys, String ArrayList filters = new ArrayList<>(); String lowestBound = null; // lower boundary (previous split in the iteration) for (int i = 0; i < splitKeys.size(); i++) { - String splitKey = splitKeys.get(i).toString(); - String rangeFilter = null; + String splitKey = splitKeys.get(i).get("_id").toString(); + String rangeFilter; if (i == 0) { // this is the first split in the list, the filter defines // the range from the beginning up to this split - rangeFilter = String.format("{ $and: [ {\"_id\":{$lte:Objectd(\"%s\")}}", + rangeFilter = String.format("{ $and: [ {\"_id\":{$lte:ObjectId(\"%s\")}}", splitKey); } else if (i == splitKeys.size() - 1) { // this is the last split in the list, the filter defines From a1a4ac0fc0376aa4c43a4357f3acc930e2b53c94 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Tue, 29 Nov 2016 09:51:12 +0200 Subject: [PATCH 032/279] [BEAM-1052] Add InputDStream id to MicrobatchSource hashcode. Done to avoid collisions between splits of different sources. --- .../runners/spark/io/MicrobatchSource.java | 20 +++++++++++++------ .../beam/runners/spark/io/SourceDStream.java | 3 ++- .../spark/stateful/StateSpecFunctions.java | 2 +- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java index 4a174aaf9b2e..565637597073 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java @@ -54,6 +54,7 @@ public class MicrobatchSource(splits.get(i), maxReadTime, 1, numRecords[i], i)); + result.add(new MicrobatchSource<>(splits.get(i), maxReadTime, 1, numRecords[i], i, sourceId)); } return result; } @@ -137,8 +140,8 @@ public Coder getCheckpointMarkCoder() { return source.getCheckpointMarkCoder(); } - public int getSplitId() { - return splitId; + public String getId() { + return sourceId + "_" + splitId; } @Override @@ -150,13 +153,18 @@ public boolean equals(Object o) { return false; } MicrobatchSource that = (MicrobatchSource) o; - + if (sourceId != that.sourceId) { + return false; + } return splitId == that.splitId; + } @Override public int hashCode() { - return splitId; + int result = sourceId; + result = 31 * result + splitId; + return result; } /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java index 4e47757dc0d3..84b247b265da 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java @@ -83,7 +83,8 @@ public SourceDStream(StreamingContext ssc, @Override public scala.Option, CheckpointMarkT>>> compute(Time validTime) { MicrobatchSource microbatchSource = new MicrobatchSource<>( - unboundedSource, boundReadDuration, initialParallelism, rateControlledMaxRecords(), -1); + unboundedSource, boundReadDuration, initialParallelism, rateControlledMaxRecords(), -1, + id()); RDD, CheckpointMarkT>> rdd = new SourceRDD.Unbounded<>( ssc().sc(), runtimeContext, microbatchSource); return scala.Option.apply(rdd); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java index 48849c2d8feb..053f4ac76fa8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java @@ -144,7 +144,7 @@ public Iterator> apply(Source source, scala.Option Date: Mon, 28 Nov 2016 16:15:55 -0800 Subject: [PATCH 033/279] Add isStateful() to DoFnSignature --- .../apache/beam/sdk/transforms/reflect/DoFnSignature.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index cd93583f9b86..07509498da78 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -105,6 +105,13 @@ public abstract class DoFnSignature { @Nullable public abstract Map onTimerMethods(); + /** + * Whether the {@link DoFn} described by this signature uses state. + */ + public boolean isStateful() { + return stateDeclarations().size() > 0; + } + static Builder builder() { return new AutoValue_DoFnSignature.Builder(); } From 31a55f407473f23a61cf6dfe42c3f6f4c7880920 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 15 Nov 2016 21:35:03 -0800 Subject: [PATCH 034/279] Reject stateful DoFn in DataflowRunner --- runners/google-cloud-dataflow-java/pom.xml | 1 + .../dataflow/DataflowPipelineTranslator.java | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index da3a4d637fe7..59276e486022 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -77,6 +77,7 @@ runnable-on-service-tests + org.apache.beam.sdk.testing.UsesStatefulParDo org.apache.beam.sdk.transforms.FlattenTest diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 9acf0713afd4..0549d5b84a55 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -69,6 +69,7 @@ import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; +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.OldDoFn; @@ -77,6 +78,7 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.AppliedCombineFn; @@ -959,6 +961,8 @@ public void translate( private void translateMultiHelper( ParDo.BoundMulti transform, TranslationContext context) { + rejectStatefulDoFn(transform.getNewFn()); + context.addStep(transform, "ParallelDo"); translateInputs(context.getInput(transform), transform.getSideInputs(), context); BiMap> outputMap = @@ -987,6 +991,8 @@ public void translate( private void translateSingleHelper( ParDo.Bound transform, TranslationContext context) { + rejectStatefulDoFn(transform.getNewFn()); + context.addStep(transform, "ParallelDo"); translateInputs(context.getInput(transform), transform.getSideInputs(), context); long mainOutput = context.addOutput(context.getOutput(transform)); @@ -1033,6 +1039,18 @@ private void translateHelper( registerTransformTranslator(Read.Bounded.class, new ReadTranslator()); } + private static void rejectStatefulDoFn(DoFn doFn) { + if (DoFnSignatures.getSignature(doFn.getClass()).isStateful()) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + DataflowRunner.class.getSimpleName())); + } + } + private static void translateInputs( PCollection input, List> sideInputs, From c21167ce50875fc3839b858f518aa4995ee92550 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 28 Nov 2016 10:09:39 -0800 Subject: [PATCH 035/279] Update Dataflow worker to beam-master-20161129 --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index f1d41f235247..b629d65ed3ad 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -212,9 +212,9 @@ public class DataflowRunner extends PipelineRunner { // Default Docker container images that execute Dataflow worker harness, residing in Google // Container Registry, separately for Batch and Streaming. public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161031"; + "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161129"; public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161031"; + "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161129"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; From f52ac3ec75cfec025290f174f0f0529850c2bfd9 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 15 Nov 2016 22:27:35 -0800 Subject: [PATCH 036/279] Transmit new DoFn, not OldDoFn, in Dataflow translator --- .../beam/runners/dataflow/DataflowPipelineTranslator.java | 7 +++---- .../org/apache/beam/runners/dataflow/DataflowRunner.java | 8 +++----- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 0549d5b84a55..2af2caebd353 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -72,7 +72,6 @@ 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.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -968,7 +967,7 @@ private void translateMultiHelper( BiMap> outputMap = translateOutputs(context.getOutput(transform), context); translateFn( - transform.getFn(), + transform.getNewFn(), context.getInput(transform).getWindowingStrategy(), transform.getSideInputs(), context.getInput(transform).getCoder(), @@ -997,7 +996,7 @@ private void translateSingleHelper( translateInputs(context.getInput(transform), transform.getSideInputs(), context); long mainOutput = context.addOutput(context.getOutput(transform)); translateFn( - transform.getFn(), + transform.getNewFn(), context.getInput(transform).getWindowingStrategy(), transform.getSideInputs(), context.getInput(transform).getCoder(), @@ -1075,7 +1074,7 @@ private static void translateSideInputs( } private static void translateFn( - OldDoFn fn, + DoFn fn, WindowingStrategy windowingStrategy, Iterable> sideInputs, Coder inputCoder, diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index b629d65ed3ad..ca3f0ede8aad 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -128,7 +128,6 @@ 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.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -2364,8 +2363,7 @@ public final PCollection apply(PBegin input) { * {@link PCollectionView} backend implementation. */ @Deprecated - public static class StreamingPCollectionViewWriterFn - extends OldDoFn, T> implements OldDoFn.RequiresWindowAccess { + public static class StreamingPCollectionViewWriterFn extends DoFn, T> { private final PCollectionView view; private final Coder dataCoder; @@ -2387,8 +2385,8 @@ public Coder getDataCoder() { return dataCoder; } - @Override - public void processElement(ProcessContext c) throws Exception { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow w) throws Exception { throw new UnsupportedOperationException( String.format( "%s is a marker class only and should never be executed.", getClass().getName())); From 0e49b150e83d85ae432c640da937a9497068e71b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 10 May 2016 11:39:35 -0700 Subject: [PATCH 037/279] Remove WindowedValue.valueInEmptyWindows A value in empty windows expands to no values, so it can be dropped at any time, perhaps unintentionally. This has bitten runner authors, including Spark & Dataflow. While creating such a thing in memory is not automatically problematic, it is also not really useful. So this change removes it. --- .../direct/FlattenEvaluatorFactoryTest.java | 8 ++--- .../beam/runners/dataflow/DataflowRunner.java | 10 +++--- .../apache/beam/sdk/util/WindowedValue.java | 33 +++---------------- .../beam/sdk/testing/PaneExtractorsTest.java | 2 +- .../beam/sdk/util/WindowedValueTest.java | 10 ------ 5 files changed, 14 insertions(+), 49 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index cb27fbc923b9..39c7cabd7062 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -78,9 +78,9 @@ public void testFlattenInMemoryEvaluator() throws Exception { rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1)); leftSideEvaluator.processElement( WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024))); - leftSideEvaluator.processElement(WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING)); + leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); rightSideEvaluator.processElement( - WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); rightSideEvaluator.processElement( WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096))); @@ -104,12 +104,12 @@ public void testFlattenInMemoryEvaluator() throws Exception { flattenedLeftBundle.commit(Instant.now()).getElements(), containsInAnyOrder( WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)), - WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING), + WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING), WindowedValue.valueInGlobalWindow(1))); assertThat( flattenedRightBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), + WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)), WindowedValue.valueInGlobalWindow(-1))); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index ca3f0ede8aad..03c503dcb419 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -23,7 +23,7 @@ import static com.google.common.base.Strings.isNullOrEmpty; import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName; import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; -import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows; +import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -1233,7 +1233,7 @@ public void processElement(ProcessContext c) // are at a window boundary. c.output(IsmRecord.of( ImmutableList.of(previousWindow.get()), - valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.of(), map)))); + valueInGlobalWindow(new TransformedMap<>(WindowedValueToValue.of(), map)))); map = new HashMap<>(); } @@ -1254,7 +1254,7 @@ public void processElement(ProcessContext c) // window boundary. c.output(IsmRecord.of( ImmutableList.of(previousWindow.get()), - valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.of(), map)))); + valueInGlobalWindow(new TransformedMap<>(WindowedValueToValue.of(), map)))); } } @@ -1721,7 +1721,7 @@ public void processElement(ProcessContext c) Iterable>, Iterable>>>of( ImmutableList.of(previousWindow.get()), - valueInEmptyWindows( + valueInGlobalWindow( new TransformedMap<>( IterableWithWindowedValuesToIterable.of(), resultMap)))); multimap = HashMultimap.create(); @@ -1742,7 +1742,7 @@ public void processElement(ProcessContext c) Iterable>, Iterable>>>of( ImmutableList.of(previousWindow.get()), - valueInEmptyWindows( + valueInGlobalWindow( new TransformedMap<>(IterableWithWindowedValuesToIterable.of(), resultMap)))); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java index a0b4cf526096..3251f09c2d32 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java @@ -55,8 +55,7 @@ public abstract class WindowedValue { /** - * Returns a {@code WindowedValue} with the given value, timestamp, - * and windows. + * Returns a {@code WindowedValue} with the given value, timestamp, and windows. */ public static WindowedValue of( T value, @@ -64,10 +63,10 @@ public static WindowedValue of( Collection windows, PaneInfo pane) { checkNotNull(pane); + checkArgument( + windows.size() > 0, "Cannot create %s in no windows", WindowedValue.class.getName()); - if (windows.size() == 0 && BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) { - return valueInEmptyWindows(value, pane); - } else if (windows.size() == 1) { + if (windows.size() == 1) { return of(value, timestamp, windows.iterator().next(), pane); } else { return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, pane); @@ -122,30 +121,6 @@ public static WindowedValue timestampedValueInGlobalWindow(T value, Insta } } - /** - * Returns a {@code WindowedValue} with the given value in no windows, and the default timestamp - * and pane. - * - * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop - * it at any point, and benign runner implementation details could cause silent data loss. - */ - @Deprecated - public static WindowedValue valueInEmptyWindows(T value) { - return new ValueInEmptyWindows(value, PaneInfo.NO_FIRING); - } - - /** - * Returns a {@code WindowedValue} with the given value in no windows, and the default timestamp - * and the specified pane. - * - * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop - * it at any point, and benign runner implementation details could cause silent data loss. - */ - @Deprecated - public static WindowedValue valueInEmptyWindows(T value, PaneInfo pane) { - return new ValueInEmptyWindows(value, pane); - } - /** * Returns a new {@code WindowedValue} that is a copy of this one, but with a different value, * which may have a new type {@code NewT}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java index ef501d495c50..79106eaf4faa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java @@ -47,7 +47,7 @@ public void onlyPaneNoFiring() { PaneExtractors.onlyPane(); Iterable> noFiring = ImmutableList.of( - WindowedValue.valueInGlobalWindow(9), WindowedValue.valueInEmptyWindows(19)); + WindowedValue.valueInGlobalWindow(9), WindowedValue.valueInGlobalWindow(19)); assertThat(extractor.apply(noFiring), containsInAnyOrder(9, 19)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index 0c69a594df5a..f7656ccd6f5b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -63,15 +62,6 @@ public void testWindowedValueCoder() throws CoderException { Assert.assertArrayEquals(value.getWindows().toArray(), decodedValue.getWindows().toArray()); } - @Test - public void testExplodeWindowsInNoWindowsEmptyIterable() { - WindowedValue value = - WindowedValue.of( - "foo", Instant.now(), ImmutableList.of(), PaneInfo.NO_FIRING); - - assertThat(value.explodeWindows(), emptyIterable()); - } - @Test public void testExplodeWindowsInOneWindowEquals() { Instant now = Instant.now(); From 0123690600ebd5f83cf485c92d6a35762428cd84 Mon Sep 17 00:00:00 2001 From: sammcveety Date: Mon, 28 Nov 2016 11:26:19 -0800 Subject: [PATCH 038/279] Fix double-close bug The WritableByteChannel returned for GCS locations has a bug where calling close twice throws an Exception, so we cannot safely use AutoCloseable here. --- .../org/apache/beam/runners/dataflow/DataflowRunner.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 03c503dcb419..641daf4f5a4d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -59,7 +59,6 @@ import java.net.URL; import java.net.URLClassLoader; import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -566,15 +565,13 @@ public DataflowPipelineJob run(Pipeline pipeline) { String.format( "Location must be local or on Cloud Storage, got {}.", fileLocation)); String workSpecJson = DataflowPipelineTranslator.jobToString(newJob); - try ( - WritableByteChannel writer = - IOChannelUtils.create(fileLocation, MimeTypes.TEXT); - PrintWriter printWriter = new PrintWriter(Channels.newOutputStream(writer))) { + try (PrintWriter printWriter = new PrintWriter( + Channels.newOutputStream(IOChannelUtils.create(fileLocation, MimeTypes.TEXT)))) { printWriter.print(workSpecJson); LOG.info("Printed job specification to {}", fileLocation); } catch (IOException ex) { String error = - String.format("Cannot create output file at {}", fileLocation); + String.format("Cannot create output file at %s", fileLocation); if (isTemplate) { throw new RuntimeException(error, ex); } else { From 74682c92d3d2bc5cd5385812ce985a8a75ee4899 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Tue, 1 Nov 2016 17:58:16 -0700 Subject: [PATCH 039/279] Add a test demonstrating how to use ValueProvider with non-serializable data --- .../apache/beam/sdk/options/ValueProvider.java | 2 +- .../beam/sdk/options/ValueProviderTest.java | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java index 2f52ad476b8b..3a2e7eddf4bc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java @@ -51,7 +51,7 @@ */ @JsonSerialize(using = ValueProvider.Serializer.class) @JsonDeserialize(using = ValueProvider.Deserializer.class) -public interface ValueProvider { +public interface ValueProvider extends Serializable { /** * Return the value wrapped by this {@link ValueProvider}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java index be0f07626319..31532b944fec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.options.ValueProvider.RuntimeValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.SerializableUtils; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -250,4 +251,21 @@ public String apply(String from) { expectedException.expectMessage("Not called from a runtime context"); nvp.get(); } + + private static class NonSerializable {} + + private static class NonSerializableTranslator + implements SerializableFunction { + @Override + public NonSerializable apply(String from) { + return new NonSerializable(); + } + } + + @Test + public void testNestedValueProviderSerialize() throws Exception { + ValueProvider nvp = NestedValueProvider.of( + StaticValueProvider.of("foo"), new NonSerializableTranslator()); + SerializableUtils.ensureSerializable(nvp); + } } From 1184bfa7a3ee5d58d65c9ba9200e91f71856ce4a Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 29 Nov 2016 14:30:09 -0800 Subject: [PATCH 040/279] Remove TransformApplicationsForTesting This field is mutated but never queried. Remove Pipeline#addValueInternal This method is never called and not suitable for use. --- .../main/java/org/apache/beam/sdk/Pipeline.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index e188b35545a9..9edf4962c97d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -329,8 +327,6 @@ OutputT applyTransform(String name, InputT input, private Collection values = new ArrayList<>(); private Set usedFullNames = new HashSet<>(); private CoderRegistry coderRegistry; - private Multimap, AppliedPTransform> transformApplicationsForTesting = - HashMultimap.create(); /** * @deprecated replaced by {@link #Pipeline(PipelineRunner, PipelineOptions)} @@ -399,7 +395,6 @@ OutputT applyInternal(String name, InputT input, AppliedPTransform applied = AppliedPTransform.of( child.getFullName(), input, output, transform); - transformApplicationsForTesting.put(transform, applied); // recordAsOutput is a NOOP if already called; output.recordAsOutput(applied); verifyOutputState(output, child); @@ -513,14 +508,4 @@ private String uniquifyInternal(String namePrefix, String origName) { private String buildName(String namePrefix, String name) { return namePrefix.isEmpty() ? name : namePrefix + "/" + name; } - - /** - * Adds the given {@link PValue} to this {@link Pipeline}. - * - *

For internal use only. - */ - public void addValueInternal(PValue value) { - this.values.add(value); - LOG.debug("Adding {} to {}", value, this); - } } From 98ab559410bde425c9c1944bcd2f09293c3764dc Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 29 Nov 2016 16:57:09 -0800 Subject: [PATCH 041/279] Revert "Remove WindowedValue.valueInEmptyWindows" This reverts commit 0e49b150e83d85ae432c640da937a9497068e71b, which breaks some DataflowRunner integration tests. --- .../direct/FlattenEvaluatorFactoryTest.java | 8 ++--- .../beam/runners/dataflow/DataflowRunner.java | 10 +++--- .../apache/beam/sdk/util/WindowedValue.java | 33 ++++++++++++++++--- .../beam/sdk/testing/PaneExtractorsTest.java | 2 +- .../beam/sdk/util/WindowedValueTest.java | 10 ++++++ 5 files changed, 49 insertions(+), 14 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 39c7cabd7062..cb27fbc923b9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -78,9 +78,9 @@ public void testFlattenInMemoryEvaluator() throws Exception { rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1)); leftSideEvaluator.processElement( WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024))); - leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); + leftSideEvaluator.processElement(WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING)); rightSideEvaluator.processElement( - WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); rightSideEvaluator.processElement( WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096))); @@ -104,12 +104,12 @@ public void testFlattenInMemoryEvaluator() throws Exception { flattenedLeftBundle.commit(Instant.now()).getElements(), containsInAnyOrder( WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)), - WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING), + WindowedValue.valueInEmptyWindows(4, PaneInfo.NO_FIRING), WindowedValue.valueInGlobalWindow(1))); assertThat( flattenedRightBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), + WindowedValue.valueInEmptyWindows(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)), WindowedValue.valueInGlobalWindow(-1))); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 641daf4f5a4d..009985644155 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -23,7 +23,7 @@ import static com.google.common.base.Strings.isNullOrEmpty; import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName; import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -1230,7 +1230,7 @@ public void processElement(ProcessContext c) // are at a window boundary. c.output(IsmRecord.of( ImmutableList.of(previousWindow.get()), - valueInGlobalWindow(new TransformedMap<>(WindowedValueToValue.of(), map)))); + valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.of(), map)))); map = new HashMap<>(); } @@ -1251,7 +1251,7 @@ public void processElement(ProcessContext c) // window boundary. c.output(IsmRecord.of( ImmutableList.of(previousWindow.get()), - valueInGlobalWindow(new TransformedMap<>(WindowedValueToValue.of(), map)))); + valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.of(), map)))); } } @@ -1718,7 +1718,7 @@ public void processElement(ProcessContext c) Iterable>, Iterable>>>of( ImmutableList.of(previousWindow.get()), - valueInGlobalWindow( + valueInEmptyWindows( new TransformedMap<>( IterableWithWindowedValuesToIterable.of(), resultMap)))); multimap = HashMultimap.create(); @@ -1739,7 +1739,7 @@ public void processElement(ProcessContext c) Iterable>, Iterable>>>of( ImmutableList.of(previousWindow.get()), - valueInGlobalWindow( + valueInEmptyWindows( new TransformedMap<>(IterableWithWindowedValuesToIterable.of(), resultMap)))); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java index 3251f09c2d32..a0b4cf526096 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java @@ -55,7 +55,8 @@ public abstract class WindowedValue { /** - * Returns a {@code WindowedValue} with the given value, timestamp, and windows. + * Returns a {@code WindowedValue} with the given value, timestamp, + * and windows. */ public static WindowedValue of( T value, @@ -63,10 +64,10 @@ public static WindowedValue of( Collection windows, PaneInfo pane) { checkNotNull(pane); - checkArgument( - windows.size() > 0, "Cannot create %s in no windows", WindowedValue.class.getName()); - if (windows.size() == 1) { + if (windows.size() == 0 && BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) { + return valueInEmptyWindows(value, pane); + } else if (windows.size() == 1) { return of(value, timestamp, windows.iterator().next(), pane); } else { return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, pane); @@ -121,6 +122,30 @@ public static WindowedValue timestampedValueInGlobalWindow(T value, Insta } } + /** + * Returns a {@code WindowedValue} with the given value in no windows, and the default timestamp + * and pane. + * + * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop + * it at any point, and benign runner implementation details could cause silent data loss. + */ + @Deprecated + public static WindowedValue valueInEmptyWindows(T value) { + return new ValueInEmptyWindows(value, PaneInfo.NO_FIRING); + } + + /** + * Returns a {@code WindowedValue} with the given value in no windows, and the default timestamp + * and the specified pane. + * + * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop + * it at any point, and benign runner implementation details could cause silent data loss. + */ + @Deprecated + public static WindowedValue valueInEmptyWindows(T value, PaneInfo pane) { + return new ValueInEmptyWindows(value, pane); + } + /** * Returns a new {@code WindowedValue} that is a copy of this one, but with a different value, * which may have a new type {@code NewT}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java index 79106eaf4faa..ef501d495c50 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java @@ -47,7 +47,7 @@ public void onlyPaneNoFiring() { PaneExtractors.onlyPane(); Iterable> noFiring = ImmutableList.of( - WindowedValue.valueInGlobalWindow(9), WindowedValue.valueInGlobalWindow(19)); + WindowedValue.valueInGlobalWindow(9), WindowedValue.valueInEmptyWindows(19)); assertThat(extractor.apply(noFiring), containsInAnyOrder(9, 19)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index f7656ccd6f5b..0c69a594df5a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.util; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -62,6 +63,15 @@ public void testWindowedValueCoder() throws CoderException { Assert.assertArrayEquals(value.getWindows().toArray(), decodedValue.getWindows().toArray()); } + @Test + public void testExplodeWindowsInNoWindowsEmptyIterable() { + WindowedValue value = + WindowedValue.of( + "foo", Instant.now(), ImmutableList.of(), PaneInfo.NO_FIRING); + + assertThat(value.explodeWindows(), emptyIterable()); + } + @Test public void testExplodeWindowsInOneWindowEquals() { Instant now = Instant.now(); From d08a9f1278e0bb4ec5b08e11f6267c516c8ea56e Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 25 Oct 2016 14:57:13 -0700 Subject: [PATCH 042/279] [BEAM-747] Fix FileChecksumMatcher That Inconsistent With FS --- .../org/apache/beam/examples/WordCountIT.java | 2 +- .../beam/sdk/testing/FileChecksumMatcher.java | 168 +++++++++++++++--- .../sdk/testing/FileChecksumMatcherTest.java | 131 +++++++++++++- 3 files changed, 268 insertions(+), 33 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index f2afe6aefbda..01438de336b0 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -65,7 +65,7 @@ public void testE2EWordCount() throws Exception { "output", "results")); options.setOnSuccessMatcher( - new FileChecksumMatcher(DEFAULT_OUTPUT_CHECKSUM, options.getOutput() + "*")); + new FileChecksumMatcher(DEFAULT_OUTPUT_CHECKSUM, options.getOutput() + "*-of-*")); WordCount.main(TestPipeline.convertToArgs(options)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java index de6cea383e8c..4b249fed38c5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -19,8 +19,14 @@ package org.apache.beam.sdk.testing; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; +import com.google.common.collect.Lists; import com.google.common.hash.HashCode; import com.google.common.hash.Hashing; import com.google.common.io.CharStreams; @@ -28,14 +34,21 @@ import java.io.Reader; import java.nio.channels.Channels; import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nonnull; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,71 +56,172 @@ * Matcher to verify file checksum in E2E test. * *

For example: - *

{@code [
- *   assertTrue(job, new FileChecksumMatcher(checksumString, filePath));
- * ]}
+ *
{@code
+ *   assertThat(job, new FileChecksumMatcher(checksumString, filePath));
+ * }
+ * or + *
{@code
+ *   assertThat(job, new FileChecksumMatcher(checksumString, filePath, shardTemplate));
+ * }
+ * + *

Checksum of outputs is generated based on SHA-1 algorithm. If output file is empty, + * SHA-1 hash of empty string (da39a3ee5e6b4b0d3255bfef95601890afd80709) is used as expected. */ public class FileChecksumMatcher extends TypeSafeMatcher implements SerializableMatcher { private static final Logger LOG = LoggerFactory.getLogger(FileChecksumMatcher.class); + static final int MAX_READ_RETRIES = 4; + static final Duration DEFAULT_SLEEP_DURATION = Duration.standardSeconds(10L); + static final FluentBackoff BACK_OFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(DEFAULT_SLEEP_DURATION) + .withMaxRetries(MAX_READ_RETRIES); + + private static final Pattern DEFAULT_SHARD_TEMPLATE = + Pattern.compile("(?x) \\S* (? \\d+) -of- (? \\d+)"); + private final String expectedChecksum; private final String filePath; + private final Pattern shardTemplate; private String actualChecksum; + /** + * Constructor that uses default shard template. + * + * @param checksum expected checksum string used to verify file content. + * @param filePath path of files that's to be verified. + */ public FileChecksumMatcher(String checksum, String filePath) { + this(checksum, filePath, DEFAULT_SHARD_TEMPLATE); + } + + /** + * Constructor. + * + * @param checksum expected checksum string used to verify file content. + * @param filePath path of files that's to be verified. + * @param shardTemplate template of shard name to parse out the total number of shards + * which is used in I/O retry to avoid inconsistency of filesystem. + * Customized template should assign name "numshards" to capturing + * group - total shard number. + */ + public FileChecksumMatcher(String checksum, String filePath, Pattern shardTemplate) { checkArgument( !Strings.isNullOrEmpty(checksum), "Expected valid checksum, but received %s", checksum); checkArgument( !Strings.isNullOrEmpty(filePath), "Expected valid file path, but received %s", filePath); + checkNotNull( + shardTemplate, + "Expected non-null shard pattern. " + + "Please call the other constructor to use default pattern: %s", + DEFAULT_SHARD_TEMPLATE); this.expectedChecksum = checksum; this.filePath = filePath; + this.shardTemplate = shardTemplate; } @Override public boolean matchesSafely(PipelineResult pipelineResult) { + // Load output data + List outputs; try { - // Load output data - List outputs = readLines(filePath); + outputs = readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff()); + } catch (Exception e) { + throw new RuntimeException(String.format("Failed to read from: %s", filePath), e); + } - // Verify outputs. Checksum is computed using SHA-1 algorithm - actualChecksum = hashing(outputs); - LOG.info("Generated checksum for output data: {}", actualChecksum); + // Verify outputs. Checksum is computed using SHA-1 algorithm + actualChecksum = computeHash(outputs); + LOG.debug("Generated checksum: {}", actualChecksum); - return actualChecksum.equals(expectedChecksum); - } catch (IOException e) { - throw new RuntimeException( - String.format("Failed to read from path: %s", filePath)); - } + return actualChecksum.equals(expectedChecksum); } - private List readLines(String path) throws IOException { - List readData = new ArrayList<>(); - IOChannelFactory factory = IOChannelUtils.getFactory(path); - - // Match inputPath which may contains glob - Collection files = factory.match(path); + @VisibleForTesting + List readFilesWithRetries(Sleeper sleeper, BackOff backOff) + throws IOException, InterruptedException { + IOChannelFactory factory = IOChannelUtils.getFactory(filePath); + IOException lastException = null; + + do { + try { + // Match inputPath which may contains glob + Collection files = factory.match(filePath); + LOG.debug("Found {} file(s) by matching the path: {}", files.size(), filePath); + + if (files.isEmpty() || !checkTotalNumOfFiles(files)) { + continue; + } + + // Read data from file paths + return readLines(files, factory); + } catch (IOException e) { + // Ignore and retry + lastException = e; + LOG.warn("Error in file reading. Ignore and retry."); + } + } while(BackOffUtils.next(sleeper, backOff)); + // Failed after max retries + throw new IOException( + String.format("Unable to read file(s) after retrying %d times", MAX_READ_RETRIES), + lastException); + } - // Read data from file paths - int i = 0; + @VisibleForTesting + List readLines(Collection files, IOChannelFactory factory) throws IOException { + List allLines = Lists.newArrayList(); + int i = 1; for (String file : files) { try (Reader reader = - Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { + Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { List lines = CharStreams.readLines(reader); - readData.addAll(lines); - LOG.info( - "[{} of {}] Read {} lines from file: {}", i, files.size() - 1, lines.size(), file); + allLines.addAll(lines); + LOG.debug( + "[{} of {}] Read {} lines from file: {}", i, files.size(), lines.size(), file); } i++; } - return readData; + return allLines; + } + + /** + * Check if total number of files is correct by comparing with the number that + * is parsed from shard name using a name template. If no template is specified, + * "SSSS-of-NNNN" will be used as default, and "NNNN" will be the expected total + * number of files. + * + * @return {@code true} if at least one shard name matches template and total number + * of given files equals the number that is parsed from shard name. + */ + @VisibleForTesting + boolean checkTotalNumOfFiles(Collection files) { + for (String filePath : files) { + Path fileName = Paths.get(filePath).getFileName(); + if (fileName == null) { + // this path has zero elements + continue; + } + Matcher matcher = shardTemplate.matcher(fileName.toString()); + if (!matcher.matches()) { + // shard name doesn't match the pattern, check with the next shard + continue; + } + // once match, extract total number of shards and compare to file list + return files.size() == Integer.parseInt(matcher.group("numshards")); + } + return false; } - private String hashing(List strs) { + private String computeHash(@Nonnull List strs) { + if (strs.isEmpty()) { + return Hashing.sha1().hashString("", StandardCharsets.UTF_8).toString(); + } + List hashCodes = new ArrayList<>(); for (String str : strs) { hashCodes.add(Hashing.sha1().hashString(str, StandardCharsets.UTF_8)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java index b2f2ec853573..0dc307d528c2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -19,12 +19,20 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyCollection; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import com.google.api.client.util.BackOff; import com.google.common.io.Files; import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.regex.Pattern; + import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; import org.junit.Rule; import org.junit.Test; @@ -42,10 +50,14 @@ public class FileChecksumMatcherTest { public TemporaryFolder tmpFolder = new TemporaryFolder(); @Rule public ExpectedException thrown = ExpectedException.none(); + @Rule + public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper(); @Mock private PipelineResult pResult = Mockito.mock(PipelineResult.class); + private BackOff backOff = FileChecksumMatcher.BACK_OFF_FACTORY.backoff(); + @Test public void testPreconditionChecksumIsNull() throws IOException { String tmpPath = tmpFolder.newFile().getPath(); @@ -79,8 +91,20 @@ public void testPreconditionFilePathIsEmpty() { } @Test - public void testMatcherVerifySingleFile() throws IOException{ - File tmpFile = tmpFolder.newFile(); + public void testPreconditionShardTemplateIsNull() throws IOException { + String tmpPath = tmpFolder.newFile().getPath(); + + thrown.expect(NullPointerException.class); + thrown.expectMessage( + containsString( + "Expected non-null shard pattern. " + + "Please call the other constructor to use default pattern:")); + new FileChecksumMatcher("checksumString", tmpPath, null); + } + + @Test + public void testMatcherThatVerifiesSingleFile() throws IOException{ + File tmpFile = tmpFolder.newFile("result-000-of-001"); Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); FileChecksumMatcher matcher = new FileChecksumMatcher("a8772322f5d7b851777f820fc79d050f9d302915", tmpFile.getPath()); @@ -89,16 +113,113 @@ public void testMatcherVerifySingleFile() throws IOException{ } @Test - public void testMatcherVerifyMultipleFilesInOneDir() throws IOException { - File tmpFile1 = tmpFolder.newFile(); - File tmpFile2 = tmpFolder.newFile(); + public void testMatcherThatVerifiesMultipleFiles() throws IOException { + File tmpFile1 = tmpFolder.newFile("result-000-of-002"); + File tmpFile2 = tmpFolder.newFile("result-001-of-002"); + File tmpFile3 = tmpFolder.newFile("tmp"); Files.write("To be or not to be, ", tmpFile1, StandardCharsets.UTF_8); Files.write("it is not a question.", tmpFile2, StandardCharsets.UTF_8); + Files.write("tmp", tmpFile3, StandardCharsets.UTF_8); + FileChecksumMatcher matcher = new FileChecksumMatcher( "90552392c28396935fe4f123bd0b5c2d0f6260c8", + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "result-*")); + + assertThat(pResult, matcher); + } + + @Test + public void testMatcherThatVerifiesFileWithEmptyContent() throws IOException { + File emptyFile = tmpFolder.newFile("result-000-of-001"); + Files.write("", emptyFile, StandardCharsets.UTF_8); + FileChecksumMatcher matcher = + new FileChecksumMatcher( + "da39a3ee5e6b4b0d3255bfef95601890afd80709", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); assertThat(pResult, matcher); } + + @Test + public void testMatcherThatUsesCustomizedTemplate() throws Exception { + // Customized template: resultSSS-totalNNN + File tmpFile1 = tmpFolder.newFile("result0-total2"); + File tmpFile2 = tmpFolder.newFile("result1-total2"); + Files.write("To be or not to be, ", tmpFile1, StandardCharsets.UTF_8); + Files.write("it is not a question.", tmpFile2, StandardCharsets.UTF_8); + + Pattern customizedTemplate = + Pattern.compile("(?x) result (?\\d+) - total (?\\d+)"); + FileChecksumMatcher matcher = new FileChecksumMatcher( + "90552392c28396935fe4f123bd0b5c2d0f6260c8", + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"), + customizedTemplate); + + assertThat(pResult, matcher); + } + + @Test + public void testReadWithRetriesFailsWhenTemplateIncorrect() throws Exception { + File tmpFile = tmpFolder.newFile(); + Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); + + FileChecksumMatcher matcher = new FileChecksumMatcher( + "mock-checksum", + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"), + Pattern.compile("incorrect-template")); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); + matcher.readFilesWithRetries(fastClock, backOff); + } + + @Test + public void testReadWithRetriesFailsSinceFilesystemError() throws Exception { + File tmpFile = tmpFolder.newFile(); + Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); + + FileChecksumMatcher matcher = + spy(new FileChecksumMatcher( + "mock-checksum", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"))); + doThrow(IOException.class) + .when(matcher).readLines(anyCollection(), any(IOChannelFactory.class)); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); + matcher.readFilesWithRetries(fastClock, backOff); + } + + @Test + public void testReadWithRetriesFailsWhenOutputDirEmpty() throws Exception { + FileChecksumMatcher matcher = + new FileChecksumMatcher( + "mock-checksum", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); + matcher.readFilesWithRetries(fastClock, backOff); + } + + @Test + public void testReadWithRetriesFailsWhenRedundantFileLoaded() throws Exception { + tmpFolder.newFile("result-000-of-001"); + tmpFolder.newFile("tmp-result-000-of-001"); + + FileChecksumMatcher matcher = + new FileChecksumMatcher( + "mock-checksum", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); + matcher.readFilesWithRetries(fastClock, backOff); + } } From 4282c67c5fa4dea2fe6c8695e0ea23f383c6457b Mon Sep 17 00:00:00 2001 From: Sam Whittle Date: Thu, 10 Nov 2016 12:59:49 -0800 Subject: [PATCH 043/279] Improvements to ReduceFnRunner prefetching - add prefetch* methods for prefetching state matching existing methods - replace onTimer with batched onTimers method to allow prefetching across timers - prefetch triggers in processElements --- .../GroupAlsoByWindowViaWindowSetDoFn.java | 11 +- .../beam/runners/core/PaneInfoTracker.java | 4 + .../runners/core/ReduceFnContextFactory.java | 9 +- .../beam/runners/core/ReduceFnRunner.java | 488 +++++++++++------- .../beam/runners/core/WatermarkHold.java | 5 + .../triggers/TriggerStateMachineRunner.java | 14 +- .../beam/runners/core/ReduceFnTester.java | 4 +- .../GroupAlsoByWindowEvaluatorFactory.java | 5 +- .../beam/sdk/transforms/DoFnTester.java | 6 +- .../util/state/InMemoryTimerInternals.java | 22 +- .../beam/sdk/util/state/TimerCallback.java | 9 +- .../state/InMemoryTimerInternalsTest.java | 54 +- 12 files changed, 407 insertions(+), 224 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index 8b1081309c01..294f21d3a873 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; @@ -73,9 +72,9 @@ private GroupAlsoByWindowViaWindowSetDoFn( @Override public void processElement(ProcessContext c) throws Exception { - KeyedWorkItem element = c.element(); + KeyedWorkItem keyedWorkItem = c.element(); - K key = c.element().key(); + K key = keyedWorkItem.key(); TimerInternals timerInternals = c.windowingInternals().timerInternals(); StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); @@ -93,10 +92,8 @@ public void processElement(ProcessContext c) throws Exception { reduceFn, c.getPipelineOptions()); - reduceFnRunner.processElements(element.elementsIterable()); - for (TimerData timer : element.timersIterable()) { - reduceFnRunner.onTimer(timer); - } + reduceFnRunner.processElements(keyedWorkItem.elementsIterable()); + reduceFnRunner.onTimers(keyedWorkItem.timersIterable()); reduceFnRunner.persist(); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 8140243b342b..69a4cfd5b9ef 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -54,6 +54,10 @@ public void clear(StateAccessor state) { state.access(PANE_INFO_TAG).clear(); } + public void prefetchPaneInfo(ReduceFn.Context context) { + context.state().access(PaneInfoTracker.PANE_INFO_TAG).readLater(); + } + /** * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane * info includes the timing for the pane, who's calculation is quite subtle. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 539126a8be0d..c5bda9b46e91 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -37,7 +37,6 @@ import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.MergingStateAccessor; -import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateContext; @@ -117,7 +116,7 @@ public ReduceFn.ProcessValueContext forValue( } public ReduceFn.OnTriggerContext forTrigger(W window, - ReadableState pane, StateStyle style, OnTriggerCallbacks callbacks) { + PaneInfo pane, StateStyle style, OnTriggerCallbacks callbacks) { return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks); } @@ -389,11 +388,11 @@ public Timers timers() { private class OnTriggerContextImpl extends ReduceFn.OnTriggerContext { private final StateAccessorImpl state; - private final ReadableState pane; + private final PaneInfo pane; private final OnTriggerCallbacks callbacks; private final TimersImpl timers; - private OnTriggerContextImpl(StateAccessorImpl state, ReadableState pane, + private OnTriggerContextImpl(StateAccessorImpl state, PaneInfo pane, OnTriggerCallbacks callbacks) { reduceFn.super(); this.state = state; @@ -424,7 +423,7 @@ public StateAccessor state() { @Override public PaneInfo paneInfo() { - return pane.read(); + return pane; } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index a686f46b63cb..3a82be9b87b5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -21,12 +21,15 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableSet; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -58,7 +61,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.util.state.TimerCallback; @@ -268,6 +270,32 @@ boolean hasNoActiveWindows() { return activeWindows.getActiveAndNewWindows().isEmpty(); } + private Set openWindows(Collection windows) { + Set result = new HashSet<>(); + for (W window : windows) { + ReduceFn.Context directContext = contextFactory.base( + window, StateStyle.DIRECT); + if (!triggerRunner.isClosed(directContext.state())) { + result.add(window); + } + } + return result; + } + + private Collection windowsThatShouldFire(Set windows) throws Exception { + Collection result = new ArrayList<>(); + // Filter out timers that didn't trigger. + for (W window : windows) { + ReduceFn.Context directContext = + contextFactory.base(window, StateStyle.DIRECT); + if (triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + result.add(window); + } + } + return result; + } + /** * Incorporate {@code values} into the underlying reduce function, and manage holds, timers, * triggers, and window merging. @@ -293,25 +321,54 @@ boolean hasNoActiveWindows() { * */ public void processElements(Iterable> values) throws Exception { + if (!values.iterator().hasNext()) { + return; + } + + // Determine all the windows for elements. + Set windows = collectWindows(values); // If an incoming element introduces a new window, attempt to merge it into an existing // window eagerly. - Map windowToMergeResult = collectAndMergeWindows(values); + Map windowToMergeResult = mergeWindows(windows); + if (!windowToMergeResult.isEmpty()) { + // Update windows by removing all windows that were merged away and adding + // the windows they were merged to. We add after completing all the + // removals to avoid removing a window that was also added. + List addedWindows = new ArrayList<>(windowToMergeResult.size()); + for (Map.Entry entry : windowToMergeResult.entrySet()) { + windows.remove(entry.getKey()); + addedWindows.add(entry.getValue()); + } + windows.addAll(addedWindows); + } - Set windowsToConsider = new HashSet<>(); + prefetchWindowsForValues(windows); + + // All windows that are open before element processing may need to fire. + Set windowsToConsider = openWindows(windows); // Process each element, using the updated activeWindows determined by collectAndMergeWindows. for (WindowedValue value : values) { - windowsToConsider.addAll(processElement(windowToMergeResult, value)); + processElement(windowToMergeResult, value); } - // Trigger output from any window for which the trigger is ready + // Now that we've processed the elements, see if any of the windows need to fire. + // Prefetch state necessary to determine if the triggers should fire. for (W mergedWindow : windowsToConsider) { - ReduceFn.Context directContext = - contextFactory.base(mergedWindow, StateStyle.DIRECT); - ReduceFn.Context renamedContext = - contextFactory.base(mergedWindow, StateStyle.RENAMED); - triggerRunner.prefetchShouldFire(mergedWindow, directContext.state()); - emitIfAppropriate(directContext, renamedContext); + triggerRunner.prefetchShouldFire( + mergedWindow, contextFactory.base(mergedWindow, StateStyle.DIRECT).state()); + } + // Filter to windows that are firing. + Collection windowsToFire = windowsThatShouldFire(windowsToConsider); + // Prefetch windows that are firing. + for (W window : windowsToFire) { + prefetchEmit(contextFactory.base(window, StateStyle.DIRECT), + contextFactory.base(window, StateStyle.RENAMED)); + } + // Trigger output from firing windows. + for (W window : windowsToFire) { + emit(contextFactory.base(window, StateStyle.DIRECT), + contextFactory.base(window, StateStyle.RENAMED)); } // We're all done with merging and emitting elements so can compress the activeWindow state. @@ -325,52 +382,61 @@ public void persist() { } /** - * Extract the windows associated with the values, and invoke merge. Return a map - * from windows to the merge result window. If a window is not in the domain of - * the result map then it did not get merged into a different window. + * Extract the windows associated with the values. */ - private Map collectAndMergeWindows(Iterable> values) - throws Exception { - // No-op if no merging can take place + private Set collectWindows(Iterable> values) throws Exception { + Set windows = new HashSet<>(); + for (WindowedValue value : values) { + for (BoundedWindow untypedWindow : value.getWindows()) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + windows.add(window); + } + } + return windows; + } + + /** + * Invoke merge for the given windows and return a map from windows to the + * merge result window. Windows that were not merged are not present in the + * map. + */ + private Map mergeWindows(Set windows) throws Exception { if (windowingStrategy.getWindowFn().isNonMerging()) { - return ImmutableMap.of(); + // Return an empty map, indicating that every window is not merged. + return Collections.emptyMap(); } + Map windowToMergeResult = new HashMap<>(); // Collect the windows from all elements (except those which are too late) and // make sure they are already in the active window set or are added as NEW windows. - for (WindowedValue value : values) { - for (BoundedWindow untypedWindow : value.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - - // For backwards compat with pre 1.4 only. - // We may still have ACTIVE windows with multiple state addresses, representing - // a window who's state has not yet been eagerly merged. - // We'll go ahead and merge that state now so that we don't have to worry about - // this legacy case anywhere else. - if (activeWindows.isActive(window)) { - Set stateAddressWindows = activeWindows.readStateAddresses(window); - if (stateAddressWindows.size() > 1) { - // This is a legacy window who's state has not been eagerly merged. - // Do that now. - ReduceFn.OnMergeContext premergeContext = - contextFactory.forPremerge(window); - reduceFn.onMerge(premergeContext); - watermarkHold.onMerge(premergeContext); - activeWindows.merged(window); - } + for (W window : windows) { + // For backwards compat with pre 1.4 only. + // We may still have ACTIVE windows with multiple state addresses, representing + // a window who's state has not yet been eagerly merged. + // We'll go ahead and merge that state now so that we don't have to worry about + // this legacy case anywhere else. + if (activeWindows.isActive(window)) { + Set stateAddressWindows = activeWindows.readStateAddresses(window); + if (stateAddressWindows.size() > 1) { + // This is a legacy window who's state has not been eagerly merged. + // Do that now. + ReduceFn.OnMergeContext premergeContext = + contextFactory.forPremerge(window); + reduceFn.onMerge(premergeContext); + watermarkHold.onMerge(premergeContext); + activeWindows.merged(window); } - - // Add this window as NEW if it is not currently ACTIVE. - // If we had already seen this window and closed its trigger, then the - // window will not be currently ACTIVE. It will then be added as NEW here, - // and fall into the merging logic as usual. - activeWindows.ensureWindowExists(window); } + + // Add this window as NEW if it is not currently ACTIVE. + // If we had already seen this window and closed its trigger, then the + // window will not be currently ACTIVE. It will then be added as NEW here, + // and fall into the merging logic as usual. + activeWindows.ensureWindowExists(window); } // Merge all of the active windows and retain a mapping from source windows to result windows. - Map windowToMergeResult = new HashMap<>(); activeWindows.merge(new OnMergeCallback(windowToMergeResult)); return windowToMergeResult; } @@ -472,38 +538,50 @@ public void onMerge(Collection toBeMerged, W mergeResult) throws Exception { } /** - * Process an element. - * - * @param value the value being processed - * @return the set of windows in which the element was actually processed + * Redirect element windows to the ACTIVE windows they have been merged into. + * The compressed representation (value, {window1, window2, ...}) actually represents + * distinct elements (value, window1), (value, window2), ... + * so if window1 and window2 merge, the resulting window will contain both copies + * of the value. */ - private Collection processElement(Map windowToMergeResult, WindowedValue value) - throws Exception { - // Redirect element windows to the ACTIVE windows they have been merged into. - // The compressed representation (value, {window1, window2, ...}) actually represents - // distinct elements (value, window1), (value, window2), ... - // so if window1 and window2 merge, the resulting window will contain both copies - // of the value. - Collection windows = new ArrayList<>(); - for (BoundedWindow untypedWindow : value.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - W mergeResult = windowToMergeResult.get(window); - if (mergeResult == null) { - mergeResult = window; - } - windows.add(mergeResult); - } + private ImmutableSet toMergedWindows(final Map windowToMergeResult, + final Collection windows) { + return ImmutableSet.copyOf( + FluentIterable.from(windows).transform( + new Function() { + @Override + public W apply(BoundedWindow untypedWindow) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + W mergedWindow = windowToMergeResult.get(window); + // If the element is not present in the map, the window is unmerged. + return (mergedWindow == null) ? window : mergedWindow; + } + } + )); + } + private void prefetchWindowsForValues(Collection windows) { // Prefetch in each of the windows if we're going to need to process triggers for (W window : windows) { - ReduceFn.ProcessValueContext directContext = contextFactory.forValue( - window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT); + ReduceFn.Context directContext = contextFactory.base( + window, StateStyle.DIRECT); triggerRunner.prefetchForValue(window, directContext.state()); } + } + + /** + * Process an element. + * + * @param windowToMergeResult map of windows to merged windows. If a window is + * not present it is unmerged. + * @param value the value being processed + */ + private void processElement(Map windowToMergeResult, WindowedValue value) + throws Exception { + ImmutableSet windows = toMergedWindows(windowToMergeResult, value.getWindows()); // Process the element for each (mergeResultWindow, not closed) window it belongs to. - List triggerableWindows = new ArrayList<>(windows.size()); for (W window : windows) { ReduceFn.ProcessValueContext directContext = contextFactory.forValue( window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT); @@ -518,7 +596,6 @@ private Collection processElement(Map windowToMergeResult, WindowedValu continue; } - triggerableWindows.add(window); activeWindows.ensureWindowIsActive(window); ReduceFn.ProcessValueContext renamedContext = contextFactory.forValue( window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED); @@ -562,102 +639,152 @@ private Collection processElement(Map windowToMergeResult, WindowedValu // cannot take a trigger state from firing to non-firing. // (We don't actually assert this since it is too slow.) } - - return triggerableWindows; } /** - * Called when an end-of-window, garbage collection, or trigger-specific timer fires. + * Enriches TimerData with state necessary for processing a timer as well as + * common queries about a timer. */ - public void onTimer(TimerData timer) throws Exception { - // Which window is the timer for? - checkArgument(timer.getNamespace() instanceof WindowNamespace, - "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); - @SuppressWarnings("unchecked") - WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); - W window = windowNamespace.getWindow(); - ReduceFn.Context directContext = - contextFactory.base(window, StateStyle.DIRECT); - ReduceFn.Context renamedContext = - contextFactory.base(window, StateStyle.RENAMED); + private class EnrichedTimerData { + public final Instant timestamp; + public final ReduceFn.Context directContext; + public final ReduceFn.Context renamedContext; + // If this is an end-of-window timer then we may need to set a garbage collection timer + // if allowed lateness is non-zero. + public final boolean isEndOfWindow; + // If this is a garbage collection timer then we should trigger and + // garbage collect the window. We'll consider any timer at or after the + // end-of-window time to be a signal to garbage collect. + public final boolean isGarbageCollection; + + EnrichedTimerData( + TimerData timer, + ReduceFn.Context directContext, + ReduceFn.Context renamedContext) { + this.timestamp = timer.getTimestamp(); + this.directContext = directContext; + this.renamedContext = renamedContext; + W window = directContext.window(); + this.isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain() + && timer.getTimestamp().equals(window.maxTimestamp()); + Instant cleanupTime = garbageCollectionTime(window); + this.isGarbageCollection = !timer.getTimestamp().isBefore(cleanupTime); + } // Has this window had its trigger finish? // - The trigger may implement isClosed as constant false. // - If the window function does not support windowing then all windows will be considered // active. // So we must take conjunction of activeWindows and triggerRunner state. - boolean windowIsActiveAndOpen = - activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state()); + public boolean windowIsActiveAndOpen() { + return activeWindows.isActive(directContext.window()) + && !triggerRunner.isClosed(directContext.state()); + } + } - if (!windowIsActiveAndOpen) { - WindowTracing.debug( - "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window); + public void onTimers(Iterable timers) throws Exception { + if (!timers.iterator().hasNext()) { + return; } - // If this is an end-of-window timer then we may need to set a garbage collection timer - // if allowed lateness is non-zero. - boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain() - && timer.getTimestamp().equals(window.maxTimestamp()); - - // If this is a garbage collection timer then we should trigger and garbage collect the window. - // We'll consider any timer at or after the end-of-window time to be a signal to garbage - // collect. - Instant cleanupTime = garbageCollectionTime(window); - boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain() - && !timer.getTimestamp().isBefore(cleanupTime); - - if (isGarbageCollection) { - WindowTracing.debug( - "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - - if (windowIsActiveAndOpen) { - // We need to call onTrigger to emit the final pane if required. - // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted, - // and the watermark has passed the end of the window. - @Nullable Instant newHold = - onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow); - checkState(newHold == null, - "Hold placed at %s despite isFinished being true.", newHold); + // Create a reusable context for each timer and begin prefetching necessary + // state. + List enrichedTimers = new LinkedList(); + for (TimerData timer : timers) { + checkArgument(timer.getNamespace() instanceof WindowNamespace, + "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); + @SuppressWarnings("unchecked") + WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); + W window = windowNamespace.getWindow(); + ReduceFn.Context directContext = + contextFactory.base(window, StateStyle.DIRECT); + ReduceFn.Context renamedContext = + contextFactory.base(window, StateStyle.RENAMED); + EnrichedTimerData enrichedTimer = new EnrichedTimerData(timer, directContext, renamedContext); + enrichedTimers.add(enrichedTimer); + + // Perform prefetching of state to determine if the trigger should fire. + if (enrichedTimer.isGarbageCollection) { + triggerRunner.prefetchIsClosed(directContext.state()); + } else { + triggerRunner.prefetchShouldFire(directContext.window(), directContext.state()); } + } - // Cleanup flavor B: Clear all the remaining state for this window since we'll never - // see elements for it again. - clearAllState(directContext, renamedContext, windowIsActiveAndOpen); - } else { - WindowTracing.debug( - "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - if (windowIsActiveAndOpen) { - emitIfAppropriate(directContext, renamedContext); + // For those windows that are active and open, prefetch the triggering or emitting state. + for (EnrichedTimerData timer : enrichedTimers) { + if (timer.windowIsActiveAndOpen()) { + ReduceFn.Context directContext = timer.directContext; + if (timer.isGarbageCollection) { + prefetchOnTrigger(directContext, timer.renamedContext); + } else if (triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + prefetchEmit(directContext, timer.renamedContext); + } } + } - if (isEndOfWindow) { - // If the window strategy trigger includes a watermark trigger then at this point - // there should be no data holds, either because we'd already cleared them on an - // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate. - // We could assert this but it is very expensive. - - // Since we are processing an on-time firing we should schedule the garbage collection - // timer. (If getAllowedLateness is zero then the timer event will be considered a - // cleanup event and handled by the above). - // Note we must do this even if the trigger is finished so that we are sure to cleanup - // any final trigger finished bits. - checkState( - windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), - "Unexpected zero getAllowedLateness"); - WindowTracing.debug( - "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), + // Perform processing now that everything is prefetched. + for (EnrichedTimerData timer : enrichedTimers) { + ReduceFn.Context directContext = timer.directContext; + ReduceFn.Context renamedContext = timer.renamedContext; + + if (timer.isGarbageCollection) { + WindowTracing.debug("ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), timer.timestamp, + timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); - checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), - "Cleanup time %s is beyond end-of-time", cleanupTime); - directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); + + boolean windowIsActiveAndOpen = timer.windowIsActiveAndOpen(); + if (windowIsActiveAndOpen) { + // We need to call onTrigger to emit the final pane if required. + // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted, + // and the watermark has passed the end of the window. + @Nullable + Instant newHold = onTrigger( + directContext, renamedContext, true /* isFinished */, timer.isEndOfWindow); + checkState(newHold == null, "Hold placed at %s despite isFinished being true.", newHold); + } + + // Cleanup flavor B: Clear all the remaining state for this window since we'll never + // see elements for it again. + clearAllState(directContext, renamedContext, windowIsActiveAndOpen); + } else { + WindowTracing.debug("ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), timer.timestamp, + timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + if (timer.windowIsActiveAndOpen() + && triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + emit(directContext, renamedContext); + } + + if (timer.isEndOfWindow) { + // If the window strategy trigger includes a watermark trigger then at this point + // there should be no data holds, either because we'd already cleared them on an + // earlier onTrigger, or because we just cleared them on the above emit. + // We could assert this but it is very expensive. + + // Since we are processing an on-time firing we should schedule the garbage collection + // timer. (If getAllowedLateness is zero then the timer event will be considered a + // cleanup event and handled by the above). + // Note we must do this even if the trigger is finished so that we are sure to cleanup + // any final trigger finished bits. + checkState(windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), + "Unexpected zero getAllowedLateness"); + Instant cleanupTime = garbageCollectionTime(directContext.window()); + WindowTracing.debug( + "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Cleanup time %s is beyond end-of-time", cleanupTime); + directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); + } } } } @@ -666,7 +793,7 @@ public void onTimer(TimerData timer) throws Exception { * Clear all the state associated with {@code context}'s window. * Should only be invoked if we know all future elements for this window will be considered * beyond allowed lateness. - * This is a superset of the clearing done by {@link #emitIfAppropriate} below since: + * This is a superset of the clearing done by {@link #emitPane} below since: *

    *
  1. We can clear the trigger finished bits since we'll never need to ask if the trigger is * closed again. @@ -692,10 +819,10 @@ private void clearAllState( } else { // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2). // For (1), if !activeWindows.isActive then the window must be merging and has been - // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired + // explicitly removed by emit. But in that case the trigger must have fired // and been closed, so this case reduces to (2). // For (2), if triggerRunner.isClosed then the trigger was fired and entered the - // closed state. In that case emitIfAppropriate will have cleared all state in + // closed state. In that case emit will have cleared all state in // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows. // We also know nonEmptyPanes must have been unconditionally cleared by the trigger. // Since the trigger fired the existing watermark holds must have been cleared, and since @@ -737,17 +864,23 @@ private boolean shouldDiscardAfterFiring(boolean isFinished) { return false; } + private void prefetchEmit(ReduceFn.Context directContext, + ReduceFn.Context renamedContext) { + triggerRunner.prefetchShouldFire(directContext.window(), directContext.state()); + triggerRunner.prefetchOnFire(directContext.window(), directContext.state()); + triggerRunner.prefetchIsClosed(directContext.state()); + prefetchOnTrigger(directContext, renamedContext); + } + /** - * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state. + * Emit if a trigger is ready to fire or timers require it, and cleanup state. */ - private void emitIfAppropriate(ReduceFn.Context directContext, + private void emit( + ReduceFn.Context directContext, ReduceFn.Context renamedContext) throws Exception { - if (!triggerRunner.shouldFire( - directContext.window(), directContext.timers(), directContext.state())) { - // Ignore unless trigger is ready to fire - return; - } + checkState(triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())); // Inform the trigger of the transition to see if it is finished triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state()); @@ -782,7 +915,7 @@ private void emitIfAppropriate(ReduceFn.Context directCon } /** - * Do we need to emit a pane? + * Do we need to emit? */ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) { if (!isEmpty) { @@ -800,6 +933,15 @@ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing return false; } + private void prefetchOnTrigger( + final ReduceFn.Context directContext, + ReduceFn.Context renamedContext) { + paneInfoTracker.prefetchPaneInfo(directContext); + watermarkHold.prefetchExtract(renamedContext); + nonEmptyPanes.isEmpty(renamedContext.state()).readLater(); + reduceFn.prefetchOnTrigger(directContext.state()); + } + /** * Run the {@link ReduceFn#onTrigger} method and produce any necessary output. * @@ -813,25 +955,17 @@ private Instant onTrigger( throws Exception { Instant inputWM = timerInternals.currentInputWatermarkTime(); - // Prefetch necessary states - ReadableState outputTimestampFuture = - watermarkHold.extractAndRelease(renamedContext, isFinished).readLater(); - ReadableState paneFuture = - paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater(); - ReadableState isEmptyFuture = - nonEmptyPanes.isEmpty(renamedContext.state()).readLater(); - - reduceFn.prefetchOnTrigger(directContext.state()); - triggerRunner.prefetchOnFire(directContext.window(), directContext.state()); - // Calculate the pane info. - final PaneInfo pane = paneFuture.read(); - // Extract the window hold, and as a side effect clear it. + final PaneInfo pane = paneInfoTracker.getNextPaneInfo(directContext, isFinished).read(); - WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read(); + // Extract the window hold, and as a side effect clear it. + final WatermarkHold.OldAndNewHolds pair = + watermarkHold.extractAndRelease(renamedContext, isFinished).read(); final Instant outputTimestamp = pair.oldHold; @Nullable Instant newHold = pair.newHold; + final boolean isEmpty = nonEmptyPanes.isEmpty(renamedContext.state()).read(); + if (newHold != null) { // We can't be finished yet. checkState( @@ -863,11 +997,11 @@ private Instant onTrigger( } // Only emit a pane if it has data or empty panes are observable. - if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) { + if (needToEmit(isEmpty, isFinished, pane.getTiming())) { // Run reduceFn.onTrigger method. final List windows = Collections.singletonList(directContext.window()); ReduceFn.OnTriggerContext renamedTriggerContext = - contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED, + contextFactory.forTrigger(directContext.window(), pane, StateStyle.RENAMED, new OnTriggerCallbacks() { @Override public void output(OutputT toOutput) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index 3c0457178634..7f1afcc0a79c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -444,6 +444,11 @@ public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) { } } + public void prefetchExtract(final ReduceFn.Context context) { + context.state().access(elementHoldTag).readLater(); + context.state().access(EXTRA_HOLD_TAG).readLater(); + } + /** * Return (a future for) the earliest hold for {@code context}. Clear all the holds after * reading, but add/restore an end-of-window or garbage collection hold if required. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java index 9f03216f5b9b..2f277eb5bce3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java @@ -99,25 +99,25 @@ public boolean isClosed(StateAccessor state) { return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger); } - public void prefetchForValue(W window, StateAccessor state) { + public void prefetchIsClosed(StateAccessor state) { if (isFinishedSetNeeded()) { state.access(FINISHED_BITS_TAG).readLater(); } + } + + public void prefetchForValue(W window, StateAccessor state) { + prefetchIsClosed(state); rootTrigger.getSpec().prefetchOnElement( contextFactory.createStateAccessor(window, rootTrigger)); } public void prefetchOnFire(W window, StateAccessor state) { - if (isFinishedSetNeeded()) { - state.access(FINISHED_BITS_TAG).readLater(); - } + prefetchIsClosed(state); rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger)); } public void prefetchShouldFire(W window, StateAccessor state) { - if (isFinishedSetNeeded()) { - state.access(FINISHED_BITS_TAG).readLater(); - } + prefetchIsClosed(state); rootTrigger.getSpec().prefetchShouldFire( contextFactory.createStateAccessor(window, rootTrigger)); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 337be23610a2..8be8ae5b00b6 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -509,8 +509,10 @@ public WindowedValue apply(TimestampedValue input) { public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception { ReduceFnRunner runner = createRunner(); - runner.onTimer( + ArrayList timers = new ArrayList(1); + timers.add( TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain)); + runner.onTimers(timers); runner.persist(); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 9d25bc6a8a7b..f70fb94a7d4c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -47,7 +47,6 @@ import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -201,9 +200,7 @@ public boolean isEmpty() { // Drop any elements within expired windows reduceFnRunner.processElements( dropExpiredWindows(key, workItem.elementsIterable(), timerInternals)); - for (TimerData timer : workItem.timersIterable()) { - reduceFnRunner.onTimer(timer); - } + reduceFnRunner.onTimers(workItem.timersIterable()); reduceFnRunner.persist(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index daa8a0600e8a..f8b12226a2b0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -466,8 +466,10 @@ public AggregateT getAggregatorValue(Aggregator agg) private static TimerCallback collectInto(final List firedTimers) { return new TimerCallback() { @Override - public void onTimer(TimerInternals.TimerData timer) throws Exception { - firedTimers.add(timer); + public void onTimers(Iterable timers) throws Exception { + for (TimerInternals.TimerData timer : timers) { + firedTimers.add(timer); + } } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index a3bb45a4cc14..f1ddaac437f0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.base.MoreObjects; +import java.util.ArrayList; import java.util.HashSet; import java.util.PriorityQueue; import java.util.Set; @@ -235,13 +236,20 @@ private void advanceAndFire( throws Exception { checkNotNull(timerCallback); PriorityQueue queue = queue(domain); - while (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { - // Remove before firing, so that if the callback adds another identical - // timer we don't remove it. - TimerData timer = queue.remove(); - WindowTracing.trace( - "InMemoryTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime); - timerCallback.onTimer(timer); + while (true) { + ArrayList firedTimers = new ArrayList(); + while (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { + // Remove before firing, so that if the callback adds another identical + // timer we don't remove it. + TimerData timer = queue.remove(); + firedTimers.add(timer); + WindowTracing.trace( + "InMemoryTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime); + } + if (firedTimers.isEmpty()) { + break; + } + timerCallback.onTimers(firedTimers); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java index 6598e300f256..dfdfd5b41672 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java @@ -19,16 +19,17 @@ import org.apache.beam.sdk.util.TimerInternals; + /** - * A callback that processes a {@link TimerInternals.TimerData TimerData}. + * A callback that processes an Iterable of {@link TimerInternals.TimerData TimerData}. */ public interface TimerCallback { - /** Processes the {@link TimerInternals.TimerData TimerData}. */ - void onTimer(TimerInternals.TimerData timer) throws Exception; + /** Processes an Iterable of {@link TimerInternals.TimerData TimerData}. */ + void onTimers(Iterable timers) throws Exception; TimerCallback NO_OP = new TimerCallback() { @Override - public void onTimer(TimerInternals.TimerData timer) throws Exception { + public void onTimers(Iterable timers) throws Exception { // Nothing } }; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java index 951803a8f659..a3a774952de5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.util.state; +import static org.mockito.Matchers.argThat; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.joda.time.Instant; @@ -24,6 +29,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -44,6 +50,37 @@ public void setUp() { MockitoAnnotations.initMocks(this); } + private static class TimersAre extends ArgumentMatcher> { + final List expectedTimers; + TimersAre(List timers) { + expectedTimers = timers; + } + + @Override + public boolean matches(Object actual) { + if (actual == null || !(actual instanceof Iterable)) { + return false; + } + @SuppressWarnings("unchecked") + Iterable timers = (Iterable) actual; + + List actualTimers = new ArrayList(); + for (TimerData timer : timers) { + actualTimers.add(timer); + } + return expectedTimers.equals(actualTimers); + } + + @Override + public String toString() { + return "ordered timers " + expectedTimers.toString(); + } + } + + private static TimersAre timersAre(TimerData... timers) { + return new TimersAre(Arrays.asList(timers)); + } + @Test public void testFiringTimers() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); @@ -54,7 +91,7 @@ public void testFiringTimers() throws Exception { underTest.setTimer(processingTime2); underTest.advanceProcessingTime(timerCallback, new Instant(20)); - Mockito.verify(timerCallback).onTimer(processingTime1); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime1))); Mockito.verifyNoMoreInteractions(timerCallback); // Advancing just a little shouldn't refire @@ -63,13 +100,13 @@ public void testFiringTimers() throws Exception { // Adding the timer and advancing a little should refire underTest.setTimer(processingTime1); - Mockito.verify(timerCallback).onTimer(processingTime1); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime1))); underTest.advanceProcessingTime(timerCallback, new Instant(21)); Mockito.verifyNoMoreInteractions(timerCallback); // And advancing the rest of the way should still have the other timer underTest.advanceProcessingTime(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimer(processingTime2); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime2))); Mockito.verifyNoMoreInteractions(timerCallback); } @@ -87,13 +124,11 @@ public void testTimerOrdering() throws Exception { underTest.setTimer(watermarkTime2); underTest.advanceInputWatermark(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimer(watermarkTime1); - Mockito.verify(timerCallback).onTimer(watermarkTime2); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(watermarkTime1, watermarkTime2))); Mockito.verifyNoMoreInteractions(timerCallback); underTest.advanceProcessingTime(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimer(processingTime1); - Mockito.verify(timerCallback).onTimer(processingTime2); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime1, processingTime2))); Mockito.verifyNoMoreInteractions(timerCallback); } @@ -107,10 +142,9 @@ public void testDeduplicate() throws Exception { underTest.setTimer(processingTime); underTest.setTimer(processingTime); underTest.advanceProcessingTime(timerCallback, new Instant(20)); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime))); underTest.advanceInputWatermark(timerCallback, new Instant(20)); - - Mockito.verify(timerCallback).onTimer(processingTime); - Mockito.verify(timerCallback).onTimer(watermarkTime); + Mockito.verify(timerCallback).onTimers(argThat(timersAre(watermarkTime))); Mockito.verifyNoMoreInteractions(timerCallback); } } From 68a2025f988a0131c6fc21649a3dbd4f71c15688 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 14 Nov 2016 14:15:35 -0800 Subject: [PATCH 044/279] Move PerKeyCombineFnRunners to runners/core. --- .../runners/core}/PerKeyCombineFnRunners.java | 13 +++++-- .../FlinkMergingNonShuffleReduceFunction.java | 2 +- .../FlinkMergingPartialReduceFunction.java | 2 +- .../functions/FlinkMergingReduceFunction.java | 2 +- .../functions/FlinkPartialReduceFunction.java | 2 +- .../functions/FlinkReduceFunction.java | 2 +- .../apache/beam/sdk/transforms/Combine.java | 37 +++++++++++++++---- 7 files changed, 43 insertions(+), 17 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/PerKeyCombineFnRunners.java (95%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java similarity index 95% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java index 35d0f2d8fe8d..6f0ff96ef485 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.collect.Iterables; import java.util.Collection; @@ -24,9 +24,11 @@ import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; -import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.CombineContextFactory; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.SideInputReader; /** * Static utility methods that provide {@link PerKeyCombineFnRunner} implementations @@ -38,12 +40,15 @@ public class PerKeyCombineFnRunners { */ public static PerKeyCombineFnRunner create(PerKeyCombineFn perKeyCombineFn) { - if (perKeyCombineFn instanceof RequiresContextInternal) { + if (perKeyCombineFn instanceof KeyedCombineFnWithContext) { return new KeyedCombineFnWithContextRunner<>( (KeyedCombineFnWithContext) perKeyCombineFn); - } else { + } else if (perKeyCombineFn instanceof KeyedCombineFn) { return new KeyedCombineFnRunner<>( (KeyedCombineFn) perKeyCombineFn); + } else { + throw new IllegalStateException( + String.format("Unknown type of CombineFn: %s", perKeyCombineFn.getClass())); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index a4284f8fa2d9..3db98a31e6ca 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -33,7 +34,6 @@ 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; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java index 30d332684b54..ea0669a94b2b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -31,7 +32,6 @@ 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; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java index 29dc1e379a89..9a4aadc83e44 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -33,7 +34,6 @@ 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; 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 3ea456aca995..ca804615e4f7 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 @@ -24,6 +24,7 @@ import java.util.Comparator; import java.util.Iterator; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -32,7 +33,6 @@ 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; 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 ab0c47199fbb..b4d003c9e8a5 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 @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -34,7 +35,6 @@ 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; 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 ac8acfc3516c..be063e2f9c84 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 @@ -42,6 +42,7 @@ 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.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase.AbstractGlobalCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.AbstractPerKeyCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; @@ -57,8 +58,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.PerKeyCombineFnRunner; -import org.apache.beam.sdk.util.PerKeyCombineFnRunners; import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowingStrategy; @@ -1940,7 +1939,7 @@ private PCollection> applyHelper(PCollection, InputT, AccumT, AccumT> hotPreCombine; PerKeyCombineFn, AccumT, OutputT> postCombine; - if (!(typedFn instanceof RequiresContextInternal)) { + if (typedFn instanceof KeyedCombineFn) { final KeyedCombineFn keyedFn = (KeyedCombineFn) typedFn; hotPreCombine = @@ -2027,7 +2026,7 @@ public void populateDisplayData(DisplayData.Builder builder) { builder.delegate(PerKeyWithHotKeyFanout.this); } }; - } else { + } else if (typedFn instanceof KeyedCombineFnWithContext) { final KeyedCombineFnWithContext keyedFnWithContext = (KeyedCombineFnWithContext) typedFn; hotPreCombine = @@ -2120,6 +2119,9 @@ public void populateDisplayData(DisplayData.Builder builder) { builder.delegate(PerKeyWithHotKeyFanout.this); } }; + } else { + throw new IllegalStateException( + String.format("Unknown type of CombineFn: %s", typedFn.getClass())); } // Use the provided hotKeyFanout fn to split into "hot" and "cold" keys, @@ -2389,15 +2391,34 @@ public List> getSideInputs() { public PCollection> apply( PCollection>> input) { - final PerKeyCombineFnRunner combineFnRunner = - PerKeyCombineFnRunners.create(fn); PCollection> output = input.apply(ParDo.of( new OldDoFn>, KV>() { @Override - public void processElement(ProcessContext c) { + public void processElement(final ProcessContext c) { K key = c.element().getKey(); - c.output(KV.of(key, combineFnRunner.apply(key, c.element().getValue(), c))); + OutputT output; + if (fn instanceof KeyedCombineFnWithContext) { + output = ((KeyedCombineFnWithContext) fn) + .apply(key, c.element().getValue(), new CombineWithContext.Context() { + @Override + public PipelineOptions getPipelineOptions() { + return c.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return c.sideInput(view); + } + }); + } else if (fn instanceof KeyedCombineFn) { + output = ((KeyedCombineFn) fn) + .apply(key, c.element().getValue()); + } else { + throw new IllegalStateException( + String.format("Unknown type of CombineFn: %s", fn.getClass())); + } + c.output(KV.of(key, output)); } @Override From aaa3b91e1e7b39dd585314a6017235cdd127e923 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 30 Nov 2016 15:21:53 -0800 Subject: [PATCH 045/279] Revert "Improvements to ReduceFnRunner prefetching" This reverts commit 4282c67c5fa4dea2fe6c8695e0ea23f383c6457b, which contained some incompatibilities outside of runners-core. --- .../GroupAlsoByWindowViaWindowSetDoFn.java | 11 +- .../beam/runners/core/PaneInfoTracker.java | 4 - .../runners/core/ReduceFnContextFactory.java | 9 +- .../beam/runners/core/ReduceFnRunner.java | 488 +++++++----------- .../beam/runners/core/WatermarkHold.java | 5 - .../triggers/TriggerStateMachineRunner.java | 14 +- .../beam/runners/core/ReduceFnTester.java | 4 +- .../GroupAlsoByWindowEvaluatorFactory.java | 5 +- .../beam/sdk/transforms/DoFnTester.java | 6 +- .../util/state/InMemoryTimerInternals.java | 22 +- .../beam/sdk/util/state/TimerCallback.java | 9 +- .../state/InMemoryTimerInternalsTest.java | 54 +- 12 files changed, 224 insertions(+), 407 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index 294f21d3a873..8b1081309c01 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; @@ -72,9 +73,9 @@ private GroupAlsoByWindowViaWindowSetDoFn( @Override public void processElement(ProcessContext c) throws Exception { - KeyedWorkItem keyedWorkItem = c.element(); + KeyedWorkItem element = c.element(); - K key = keyedWorkItem.key(); + K key = c.element().key(); TimerInternals timerInternals = c.windowingInternals().timerInternals(); StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); @@ -92,8 +93,10 @@ public void processElement(ProcessContext c) throws Exception { reduceFn, c.getPipelineOptions()); - reduceFnRunner.processElements(keyedWorkItem.elementsIterable()); - reduceFnRunner.onTimers(keyedWorkItem.timersIterable()); + reduceFnRunner.processElements(element.elementsIterable()); + for (TimerData timer : element.timersIterable()) { + reduceFnRunner.onTimer(timer); + } reduceFnRunner.persist(); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 69a4cfd5b9ef..8140243b342b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -54,10 +54,6 @@ public void clear(StateAccessor state) { state.access(PANE_INFO_TAG).clear(); } - public void prefetchPaneInfo(ReduceFn.Context context) { - context.state().access(PaneInfoTracker.PANE_INFO_TAG).readLater(); - } - /** * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane * info includes the timing for the pane, who's calculation is quite subtle. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index c5bda9b46e91..539126a8be0d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -37,6 +37,7 @@ import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.MergingStateAccessor; +import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateContext; @@ -116,7 +117,7 @@ public ReduceFn.ProcessValueContext forValue( } public ReduceFn.OnTriggerContext forTrigger(W window, - PaneInfo pane, StateStyle style, OnTriggerCallbacks callbacks) { + ReadableState pane, StateStyle style, OnTriggerCallbacks callbacks) { return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks); } @@ -388,11 +389,11 @@ public Timers timers() { private class OnTriggerContextImpl extends ReduceFn.OnTriggerContext { private final StateAccessorImpl state; - private final PaneInfo pane; + private final ReadableState pane; private final OnTriggerCallbacks callbacks; private final TimersImpl timers; - private OnTriggerContextImpl(StateAccessorImpl state, PaneInfo pane, + private OnTriggerContextImpl(StateAccessorImpl state, ReadableState pane, OnTriggerCallbacks callbacks) { reduceFn.super(); this.state = state; @@ -423,7 +424,7 @@ public StateAccessor state() { @Override public PaneInfo paneInfo() { - return pane; + return pane.read(); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index 3a82be9b87b5..a686f46b63cb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -21,15 +21,12 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableSet; +import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -61,6 +58,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; +import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.util.state.TimerCallback; @@ -270,32 +268,6 @@ boolean hasNoActiveWindows() { return activeWindows.getActiveAndNewWindows().isEmpty(); } - private Set openWindows(Collection windows) { - Set result = new HashSet<>(); - for (W window : windows) { - ReduceFn.Context directContext = contextFactory.base( - window, StateStyle.DIRECT); - if (!triggerRunner.isClosed(directContext.state())) { - result.add(window); - } - } - return result; - } - - private Collection windowsThatShouldFire(Set windows) throws Exception { - Collection result = new ArrayList<>(); - // Filter out timers that didn't trigger. - for (W window : windows) { - ReduceFn.Context directContext = - contextFactory.base(window, StateStyle.DIRECT); - if (triggerRunner.shouldFire( - directContext.window(), directContext.timers(), directContext.state())) { - result.add(window); - } - } - return result; - } - /** * Incorporate {@code values} into the underlying reduce function, and manage holds, timers, * triggers, and window merging. @@ -321,54 +293,25 @@ private Collection windowsThatShouldFire(Set windows) throws Exception { *
*/ public void processElements(Iterable> values) throws Exception { - if (!values.iterator().hasNext()) { - return; - } - - // Determine all the windows for elements. - Set windows = collectWindows(values); // If an incoming element introduces a new window, attempt to merge it into an existing // window eagerly. - Map windowToMergeResult = mergeWindows(windows); - if (!windowToMergeResult.isEmpty()) { - // Update windows by removing all windows that were merged away and adding - // the windows they were merged to. We add after completing all the - // removals to avoid removing a window that was also added. - List addedWindows = new ArrayList<>(windowToMergeResult.size()); - for (Map.Entry entry : windowToMergeResult.entrySet()) { - windows.remove(entry.getKey()); - addedWindows.add(entry.getValue()); - } - windows.addAll(addedWindows); - } + Map windowToMergeResult = collectAndMergeWindows(values); - prefetchWindowsForValues(windows); - - // All windows that are open before element processing may need to fire. - Set windowsToConsider = openWindows(windows); + Set windowsToConsider = new HashSet<>(); // Process each element, using the updated activeWindows determined by collectAndMergeWindows. for (WindowedValue value : values) { - processElement(windowToMergeResult, value); + windowsToConsider.addAll(processElement(windowToMergeResult, value)); } - // Now that we've processed the elements, see if any of the windows need to fire. - // Prefetch state necessary to determine if the triggers should fire. + // Trigger output from any window for which the trigger is ready for (W mergedWindow : windowsToConsider) { - triggerRunner.prefetchShouldFire( - mergedWindow, contextFactory.base(mergedWindow, StateStyle.DIRECT).state()); - } - // Filter to windows that are firing. - Collection windowsToFire = windowsThatShouldFire(windowsToConsider); - // Prefetch windows that are firing. - for (W window : windowsToFire) { - prefetchEmit(contextFactory.base(window, StateStyle.DIRECT), - contextFactory.base(window, StateStyle.RENAMED)); - } - // Trigger output from firing windows. - for (W window : windowsToFire) { - emit(contextFactory.base(window, StateStyle.DIRECT), - contextFactory.base(window, StateStyle.RENAMED)); + ReduceFn.Context directContext = + contextFactory.base(mergedWindow, StateStyle.DIRECT); + ReduceFn.Context renamedContext = + contextFactory.base(mergedWindow, StateStyle.RENAMED); + triggerRunner.prefetchShouldFire(mergedWindow, directContext.state()); + emitIfAppropriate(directContext, renamedContext); } // We're all done with merging and emitting elements so can compress the activeWindow state. @@ -382,61 +325,52 @@ public void persist() { } /** - * Extract the windows associated with the values. + * Extract the windows associated with the values, and invoke merge. Return a map + * from windows to the merge result window. If a window is not in the domain of + * the result map then it did not get merged into a different window. */ - private Set collectWindows(Iterable> values) throws Exception { - Set windows = new HashSet<>(); - for (WindowedValue value : values) { - for (BoundedWindow untypedWindow : value.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - windows.add(window); - } - } - return windows; - } - - /** - * Invoke merge for the given windows and return a map from windows to the - * merge result window. Windows that were not merged are not present in the - * map. - */ - private Map mergeWindows(Set windows) throws Exception { + private Map collectAndMergeWindows(Iterable> values) + throws Exception { + // No-op if no merging can take place if (windowingStrategy.getWindowFn().isNonMerging()) { - // Return an empty map, indicating that every window is not merged. - return Collections.emptyMap(); + return ImmutableMap.of(); } - Map windowToMergeResult = new HashMap<>(); // Collect the windows from all elements (except those which are too late) and // make sure they are already in the active window set or are added as NEW windows. - for (W window : windows) { - // For backwards compat with pre 1.4 only. - // We may still have ACTIVE windows with multiple state addresses, representing - // a window who's state has not yet been eagerly merged. - // We'll go ahead and merge that state now so that we don't have to worry about - // this legacy case anywhere else. - if (activeWindows.isActive(window)) { - Set stateAddressWindows = activeWindows.readStateAddresses(window); - if (stateAddressWindows.size() > 1) { - // This is a legacy window who's state has not been eagerly merged. - // Do that now. - ReduceFn.OnMergeContext premergeContext = - contextFactory.forPremerge(window); - reduceFn.onMerge(premergeContext); - watermarkHold.onMerge(premergeContext); - activeWindows.merged(window); + for (WindowedValue value : values) { + for (BoundedWindow untypedWindow : value.getWindows()) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + + // For backwards compat with pre 1.4 only. + // We may still have ACTIVE windows with multiple state addresses, representing + // a window who's state has not yet been eagerly merged. + // We'll go ahead and merge that state now so that we don't have to worry about + // this legacy case anywhere else. + if (activeWindows.isActive(window)) { + Set stateAddressWindows = activeWindows.readStateAddresses(window); + if (stateAddressWindows.size() > 1) { + // This is a legacy window who's state has not been eagerly merged. + // Do that now. + ReduceFn.OnMergeContext premergeContext = + contextFactory.forPremerge(window); + reduceFn.onMerge(premergeContext); + watermarkHold.onMerge(premergeContext); + activeWindows.merged(window); + } } - } - // Add this window as NEW if it is not currently ACTIVE. - // If we had already seen this window and closed its trigger, then the - // window will not be currently ACTIVE. It will then be added as NEW here, - // and fall into the merging logic as usual. - activeWindows.ensureWindowExists(window); + // Add this window as NEW if it is not currently ACTIVE. + // If we had already seen this window and closed its trigger, then the + // window will not be currently ACTIVE. It will then be added as NEW here, + // and fall into the merging logic as usual. + activeWindows.ensureWindowExists(window); + } } // Merge all of the active windows and retain a mapping from source windows to result windows. + Map windowToMergeResult = new HashMap<>(); activeWindows.merge(new OnMergeCallback(windowToMergeResult)); return windowToMergeResult; } @@ -538,50 +472,38 @@ public void onMerge(Collection toBeMerged, W mergeResult) throws Exception { } /** - * Redirect element windows to the ACTIVE windows they have been merged into. - * The compressed representation (value, {window1, window2, ...}) actually represents - * distinct elements (value, window1), (value, window2), ... - * so if window1 and window2 merge, the resulting window will contain both copies - * of the value. + * Process an element. + * + * @param value the value being processed + * @return the set of windows in which the element was actually processed */ - private ImmutableSet toMergedWindows(final Map windowToMergeResult, - final Collection windows) { - return ImmutableSet.copyOf( - FluentIterable.from(windows).transform( - new Function() { - @Override - public W apply(BoundedWindow untypedWindow) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - W mergedWindow = windowToMergeResult.get(window); - // If the element is not present in the map, the window is unmerged. - return (mergedWindow == null) ? window : mergedWindow; - } - } - )); - } + private Collection processElement(Map windowToMergeResult, WindowedValue value) + throws Exception { + // Redirect element windows to the ACTIVE windows they have been merged into. + // The compressed representation (value, {window1, window2, ...}) actually represents + // distinct elements (value, window1), (value, window2), ... + // so if window1 and window2 merge, the resulting window will contain both copies + // of the value. + Collection windows = new ArrayList<>(); + for (BoundedWindow untypedWindow : value.getWindows()) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + W mergeResult = windowToMergeResult.get(window); + if (mergeResult == null) { + mergeResult = window; + } + windows.add(mergeResult); + } - private void prefetchWindowsForValues(Collection windows) { // Prefetch in each of the windows if we're going to need to process triggers for (W window : windows) { - ReduceFn.Context directContext = contextFactory.base( - window, StateStyle.DIRECT); + ReduceFn.ProcessValueContext directContext = contextFactory.forValue( + window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT); triggerRunner.prefetchForValue(window, directContext.state()); } - } - - /** - * Process an element. - * - * @param windowToMergeResult map of windows to merged windows. If a window is - * not present it is unmerged. - * @param value the value being processed - */ - private void processElement(Map windowToMergeResult, WindowedValue value) - throws Exception { - ImmutableSet windows = toMergedWindows(windowToMergeResult, value.getWindows()); // Process the element for each (mergeResultWindow, not closed) window it belongs to. + List triggerableWindows = new ArrayList<>(windows.size()); for (W window : windows) { ReduceFn.ProcessValueContext directContext = contextFactory.forValue( window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT); @@ -596,6 +518,7 @@ private void processElement(Map windowToMergeResult, WindowedValue continue; } + triggerableWindows.add(window); activeWindows.ensureWindowIsActive(window); ReduceFn.ProcessValueContext renamedContext = contextFactory.forValue( window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED); @@ -639,152 +562,102 @@ private void processElement(Map windowToMergeResult, WindowedValue // cannot take a trigger state from firing to non-firing. // (We don't actually assert this since it is too slow.) } + + return triggerableWindows; } /** - * Enriches TimerData with state necessary for processing a timer as well as - * common queries about a timer. + * Called when an end-of-window, garbage collection, or trigger-specific timer fires. */ - private class EnrichedTimerData { - public final Instant timestamp; - public final ReduceFn.Context directContext; - public final ReduceFn.Context renamedContext; - // If this is an end-of-window timer then we may need to set a garbage collection timer - // if allowed lateness is non-zero. - public final boolean isEndOfWindow; - // If this is a garbage collection timer then we should trigger and - // garbage collect the window. We'll consider any timer at or after the - // end-of-window time to be a signal to garbage collect. - public final boolean isGarbageCollection; - - EnrichedTimerData( - TimerData timer, - ReduceFn.Context directContext, - ReduceFn.Context renamedContext) { - this.timestamp = timer.getTimestamp(); - this.directContext = directContext; - this.renamedContext = renamedContext; - W window = directContext.window(); - this.isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain() - && timer.getTimestamp().equals(window.maxTimestamp()); - Instant cleanupTime = garbageCollectionTime(window); - this.isGarbageCollection = !timer.getTimestamp().isBefore(cleanupTime); - } + public void onTimer(TimerData timer) throws Exception { + // Which window is the timer for? + checkArgument(timer.getNamespace() instanceof WindowNamespace, + "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); + @SuppressWarnings("unchecked") + WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); + W window = windowNamespace.getWindow(); + ReduceFn.Context directContext = + contextFactory.base(window, StateStyle.DIRECT); + ReduceFn.Context renamedContext = + contextFactory.base(window, StateStyle.RENAMED); // Has this window had its trigger finish? // - The trigger may implement isClosed as constant false. // - If the window function does not support windowing then all windows will be considered // active. // So we must take conjunction of activeWindows and triggerRunner state. - public boolean windowIsActiveAndOpen() { - return activeWindows.isActive(directContext.window()) - && !triggerRunner.isClosed(directContext.state()); - } - } + boolean windowIsActiveAndOpen = + activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state()); - public void onTimers(Iterable timers) throws Exception { - if (!timers.iterator().hasNext()) { - return; + if (!windowIsActiveAndOpen) { + WindowTracing.debug( + "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window); } - // Create a reusable context for each timer and begin prefetching necessary - // state. - List enrichedTimers = new LinkedList(); - for (TimerData timer : timers) { - checkArgument(timer.getNamespace() instanceof WindowNamespace, - "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); - @SuppressWarnings("unchecked") - WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); - W window = windowNamespace.getWindow(); - ReduceFn.Context directContext = - contextFactory.base(window, StateStyle.DIRECT); - ReduceFn.Context renamedContext = - contextFactory.base(window, StateStyle.RENAMED); - EnrichedTimerData enrichedTimer = new EnrichedTimerData(timer, directContext, renamedContext); - enrichedTimers.add(enrichedTimer); - - // Perform prefetching of state to determine if the trigger should fire. - if (enrichedTimer.isGarbageCollection) { - triggerRunner.prefetchIsClosed(directContext.state()); - } else { - triggerRunner.prefetchShouldFire(directContext.window(), directContext.state()); + // If this is an end-of-window timer then we may need to set a garbage collection timer + // if allowed lateness is non-zero. + boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain() + && timer.getTimestamp().equals(window.maxTimestamp()); + + // If this is a garbage collection timer then we should trigger and garbage collect the window. + // We'll consider any timer at or after the end-of-window time to be a signal to garbage + // collect. + Instant cleanupTime = garbageCollectionTime(window); + boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain() + && !timer.getTimestamp().isBefore(cleanupTime); + + if (isGarbageCollection) { + WindowTracing.debug( + "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + + if (windowIsActiveAndOpen) { + // We need to call onTrigger to emit the final pane if required. + // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted, + // and the watermark has passed the end of the window. + @Nullable Instant newHold = + onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow); + checkState(newHold == null, + "Hold placed at %s despite isFinished being true.", newHold); } - } - // For those windows that are active and open, prefetch the triggering or emitting state. - for (EnrichedTimerData timer : enrichedTimers) { - if (timer.windowIsActiveAndOpen()) { - ReduceFn.Context directContext = timer.directContext; - if (timer.isGarbageCollection) { - prefetchOnTrigger(directContext, timer.renamedContext); - } else if (triggerRunner.shouldFire( - directContext.window(), directContext.timers(), directContext.state())) { - prefetchEmit(directContext, timer.renamedContext); - } + // Cleanup flavor B: Clear all the remaining state for this window since we'll never + // see elements for it again. + clearAllState(directContext, renamedContext, windowIsActiveAndOpen); + } else { + WindowTracing.debug( + "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + if (windowIsActiveAndOpen) { + emitIfAppropriate(directContext, renamedContext); } - } - - // Perform processing now that everything is prefetched. - for (EnrichedTimerData timer : enrichedTimers) { - ReduceFn.Context directContext = timer.directContext; - ReduceFn.Context renamedContext = timer.renamedContext; - if (timer.isGarbageCollection) { - WindowTracing.debug("ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, directContext.window(), timer.timestamp, - timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - - boolean windowIsActiveAndOpen = timer.windowIsActiveAndOpen(); - if (windowIsActiveAndOpen) { - // We need to call onTrigger to emit the final pane if required. - // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted, - // and the watermark has passed the end of the window. - @Nullable - Instant newHold = onTrigger( - directContext, renamedContext, true /* isFinished */, timer.isEndOfWindow); - checkState(newHold == null, "Hold placed at %s despite isFinished being true.", newHold); - } - - // Cleanup flavor B: Clear all the remaining state for this window since we'll never - // see elements for it again. - clearAllState(directContext, renamedContext, windowIsActiveAndOpen); - } else { - WindowTracing.debug("ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, directContext.window(), timer.timestamp, - timerInternals.currentInputWatermarkTime(), + if (isEndOfWindow) { + // If the window strategy trigger includes a watermark trigger then at this point + // there should be no data holds, either because we'd already cleared them on an + // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate. + // We could assert this but it is very expensive. + + // Since we are processing an on-time firing we should schedule the garbage collection + // timer. (If getAllowedLateness is zero then the timer event will be considered a + // cleanup event and handled by the above). + // Note we must do this even if the trigger is finished so that we are sure to cleanup + // any final trigger finished bits. + checkState( + windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), + "Unexpected zero getAllowedLateness"); + WindowTracing.debug( + "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); - if (timer.windowIsActiveAndOpen() - && triggerRunner.shouldFire( - directContext.window(), directContext.timers(), directContext.state())) { - emit(directContext, renamedContext); - } - - if (timer.isEndOfWindow) { - // If the window strategy trigger includes a watermark trigger then at this point - // there should be no data holds, either because we'd already cleared them on an - // earlier onTrigger, or because we just cleared them on the above emit. - // We could assert this but it is very expensive. - - // Since we are processing an on-time firing we should schedule the garbage collection - // timer. (If getAllowedLateness is zero then the timer event will be considered a - // cleanup event and handled by the above). - // Note we must do this even if the trigger is finished so that we are sure to cleanup - // any final trigger finished bits. - checkState(windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), - "Unexpected zero getAllowedLateness"); - Instant cleanupTime = garbageCollectionTime(directContext.window()); - WindowTracing.debug( - "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), - "Cleanup time %s is beyond end-of-time", cleanupTime); - directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); - } + checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Cleanup time %s is beyond end-of-time", cleanupTime); + directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); } } } @@ -793,7 +666,7 @@ public void onTimers(Iterable timers) throws Exception { * Clear all the state associated with {@code context}'s window. * Should only be invoked if we know all future elements for this window will be considered * beyond allowed lateness. - * This is a superset of the clearing done by {@link #emitPane} below since: + * This is a superset of the clearing done by {@link #emitIfAppropriate} below since: *
    *
  1. We can clear the trigger finished bits since we'll never need to ask if the trigger is * closed again. @@ -819,10 +692,10 @@ private void clearAllState( } else { // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2). // For (1), if !activeWindows.isActive then the window must be merging and has been - // explicitly removed by emit. But in that case the trigger must have fired + // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired // and been closed, so this case reduces to (2). // For (2), if triggerRunner.isClosed then the trigger was fired and entered the - // closed state. In that case emit will have cleared all state in + // closed state. In that case emitIfAppropriate will have cleared all state in // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows. // We also know nonEmptyPanes must have been unconditionally cleared by the trigger. // Since the trigger fired the existing watermark holds must have been cleared, and since @@ -864,23 +737,17 @@ private boolean shouldDiscardAfterFiring(boolean isFinished) { return false; } - private void prefetchEmit(ReduceFn.Context directContext, - ReduceFn.Context renamedContext) { - triggerRunner.prefetchShouldFire(directContext.window(), directContext.state()); - triggerRunner.prefetchOnFire(directContext.window(), directContext.state()); - triggerRunner.prefetchIsClosed(directContext.state()); - prefetchOnTrigger(directContext, renamedContext); - } - /** - * Emit if a trigger is ready to fire or timers require it, and cleanup state. + * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state. */ - private void emit( - ReduceFn.Context directContext, + private void emitIfAppropriate(ReduceFn.Context directContext, ReduceFn.Context renamedContext) throws Exception { - checkState(triggerRunner.shouldFire( - directContext.window(), directContext.timers(), directContext.state())); + if (!triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + // Ignore unless trigger is ready to fire + return; + } // Inform the trigger of the transition to see if it is finished triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state()); @@ -915,7 +782,7 @@ private void emit( } /** - * Do we need to emit? + * Do we need to emit a pane? */ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) { if (!isEmpty) { @@ -933,15 +800,6 @@ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing return false; } - private void prefetchOnTrigger( - final ReduceFn.Context directContext, - ReduceFn.Context renamedContext) { - paneInfoTracker.prefetchPaneInfo(directContext); - watermarkHold.prefetchExtract(renamedContext); - nonEmptyPanes.isEmpty(renamedContext.state()).readLater(); - reduceFn.prefetchOnTrigger(directContext.state()); - } - /** * Run the {@link ReduceFn#onTrigger} method and produce any necessary output. * @@ -955,17 +813,25 @@ private Instant onTrigger( throws Exception { Instant inputWM = timerInternals.currentInputWatermarkTime(); - // Calculate the pane info. - final PaneInfo pane = paneInfoTracker.getNextPaneInfo(directContext, isFinished).read(); + // Prefetch necessary states + ReadableState outputTimestampFuture = + watermarkHold.extractAndRelease(renamedContext, isFinished).readLater(); + ReadableState paneFuture = + paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater(); + ReadableState isEmptyFuture = + nonEmptyPanes.isEmpty(renamedContext.state()).readLater(); + reduceFn.prefetchOnTrigger(directContext.state()); + triggerRunner.prefetchOnFire(directContext.window(), directContext.state()); + + // Calculate the pane info. + final PaneInfo pane = paneFuture.read(); // Extract the window hold, and as a side effect clear it. - final WatermarkHold.OldAndNewHolds pair = - watermarkHold.extractAndRelease(renamedContext, isFinished).read(); + + WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read(); final Instant outputTimestamp = pair.oldHold; @Nullable Instant newHold = pair.newHold; - final boolean isEmpty = nonEmptyPanes.isEmpty(renamedContext.state()).read(); - if (newHold != null) { // We can't be finished yet. checkState( @@ -997,11 +863,11 @@ private Instant onTrigger( } // Only emit a pane if it has data or empty panes are observable. - if (needToEmit(isEmpty, isFinished, pane.getTiming())) { + if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) { // Run reduceFn.onTrigger method. final List windows = Collections.singletonList(directContext.window()); ReduceFn.OnTriggerContext renamedTriggerContext = - contextFactory.forTrigger(directContext.window(), pane, StateStyle.RENAMED, + contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED, new OnTriggerCallbacks() { @Override public void output(OutputT toOutput) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index 7f1afcc0a79c..3c0457178634 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -444,11 +444,6 @@ public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) { } } - public void prefetchExtract(final ReduceFn.Context context) { - context.state().access(elementHoldTag).readLater(); - context.state().access(EXTRA_HOLD_TAG).readLater(); - } - /** * Return (a future for) the earliest hold for {@code context}. Clear all the holds after * reading, but add/restore an end-of-window or garbage collection hold if required. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java index 2f277eb5bce3..9f03216f5b9b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java @@ -99,25 +99,25 @@ public boolean isClosed(StateAccessor state) { return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger); } - public void prefetchIsClosed(StateAccessor state) { + public void prefetchForValue(W window, StateAccessor state) { if (isFinishedSetNeeded()) { state.access(FINISHED_BITS_TAG).readLater(); } - } - - public void prefetchForValue(W window, StateAccessor state) { - prefetchIsClosed(state); rootTrigger.getSpec().prefetchOnElement( contextFactory.createStateAccessor(window, rootTrigger)); } public void prefetchOnFire(W window, StateAccessor state) { - prefetchIsClosed(state); + if (isFinishedSetNeeded()) { + state.access(FINISHED_BITS_TAG).readLater(); + } rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger)); } public void prefetchShouldFire(W window, StateAccessor state) { - prefetchIsClosed(state); + if (isFinishedSetNeeded()) { + state.access(FINISHED_BITS_TAG).readLater(); + } rootTrigger.getSpec().prefetchShouldFire( contextFactory.createStateAccessor(window, rootTrigger)); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 8be8ae5b00b6..337be23610a2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -509,10 +509,8 @@ public WindowedValue apply(TimestampedValue input) { public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception { ReduceFnRunner runner = createRunner(); - ArrayList timers = new ArrayList(1); - timers.add( + runner.onTimer( TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain)); - runner.onTimers(timers); runner.persist(); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index f70fb94a7d4c..9d25bc6a8a7b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -47,6 +47,7 @@ import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -200,7 +201,9 @@ public boolean isEmpty() { // Drop any elements within expired windows reduceFnRunner.processElements( dropExpiredWindows(key, workItem.elementsIterable(), timerInternals)); - reduceFnRunner.onTimers(workItem.timersIterable()); + for (TimerData timer : workItem.timersIterable()) { + reduceFnRunner.onTimer(timer); + } reduceFnRunner.persist(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index f8b12226a2b0..daa8a0600e8a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -466,10 +466,8 @@ public AggregateT getAggregatorValue(Aggregator agg) private static TimerCallback collectInto(final List firedTimers) { return new TimerCallback() { @Override - public void onTimers(Iterable timers) throws Exception { - for (TimerInternals.TimerData timer : timers) { - firedTimers.add(timer); - } + public void onTimer(TimerInternals.TimerData timer) throws Exception { + firedTimers.add(timer); } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index f1ddaac437f0..a3bb45a4cc14 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -21,7 +21,6 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.base.MoreObjects; -import java.util.ArrayList; import java.util.HashSet; import java.util.PriorityQueue; import java.util.Set; @@ -236,20 +235,13 @@ private void advanceAndFire( throws Exception { checkNotNull(timerCallback); PriorityQueue queue = queue(domain); - while (true) { - ArrayList firedTimers = new ArrayList(); - while (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { - // Remove before firing, so that if the callback adds another identical - // timer we don't remove it. - TimerData timer = queue.remove(); - firedTimers.add(timer); - WindowTracing.trace( - "InMemoryTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime); - } - if (firedTimers.isEmpty()) { - break; - } - timerCallback.onTimers(firedTimers); + while (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { + // Remove before firing, so that if the callback adds another identical + // timer we don't remove it. + TimerData timer = queue.remove(); + WindowTracing.trace( + "InMemoryTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime); + timerCallback.onTimer(timer); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java index dfdfd5b41672..6598e300f256 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java @@ -19,17 +19,16 @@ import org.apache.beam.sdk.util.TimerInternals; - /** - * A callback that processes an Iterable of {@link TimerInternals.TimerData TimerData}. + * A callback that processes a {@link TimerInternals.TimerData TimerData}. */ public interface TimerCallback { - /** Processes an Iterable of {@link TimerInternals.TimerData TimerData}. */ - void onTimers(Iterable timers) throws Exception; + /** Processes the {@link TimerInternals.TimerData TimerData}. */ + void onTimer(TimerInternals.TimerData timer) throws Exception; TimerCallback NO_OP = new TimerCallback() { @Override - public void onTimers(Iterable timers) throws Exception { + public void onTimer(TimerInternals.TimerData timer) throws Exception { // Nothing } }; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java index a3a774952de5..951803a8f659 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java @@ -17,11 +17,6 @@ */ package org.apache.beam.sdk.util.state; -import static org.mockito.Matchers.argThat; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.joda.time.Instant; @@ -29,7 +24,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.mockito.ArgumentMatcher; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -50,37 +44,6 @@ public void setUp() { MockitoAnnotations.initMocks(this); } - private static class TimersAre extends ArgumentMatcher> { - final List expectedTimers; - TimersAre(List timers) { - expectedTimers = timers; - } - - @Override - public boolean matches(Object actual) { - if (actual == null || !(actual instanceof Iterable)) { - return false; - } - @SuppressWarnings("unchecked") - Iterable timers = (Iterable) actual; - - List actualTimers = new ArrayList(); - for (TimerData timer : timers) { - actualTimers.add(timer); - } - return expectedTimers.equals(actualTimers); - } - - @Override - public String toString() { - return "ordered timers " + expectedTimers.toString(); - } - } - - private static TimersAre timersAre(TimerData... timers) { - return new TimersAre(Arrays.asList(timers)); - } - @Test public void testFiringTimers() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); @@ -91,7 +54,7 @@ public void testFiringTimers() throws Exception { underTest.setTimer(processingTime2); underTest.advanceProcessingTime(timerCallback, new Instant(20)); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime1))); + Mockito.verify(timerCallback).onTimer(processingTime1); Mockito.verifyNoMoreInteractions(timerCallback); // Advancing just a little shouldn't refire @@ -100,13 +63,13 @@ public void testFiringTimers() throws Exception { // Adding the timer and advancing a little should refire underTest.setTimer(processingTime1); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime1))); + Mockito.verify(timerCallback).onTimer(processingTime1); underTest.advanceProcessingTime(timerCallback, new Instant(21)); Mockito.verifyNoMoreInteractions(timerCallback); // And advancing the rest of the way should still have the other timer underTest.advanceProcessingTime(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime2))); + Mockito.verify(timerCallback).onTimer(processingTime2); Mockito.verifyNoMoreInteractions(timerCallback); } @@ -124,11 +87,13 @@ public void testTimerOrdering() throws Exception { underTest.setTimer(watermarkTime2); underTest.advanceInputWatermark(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(watermarkTime1, watermarkTime2))); + Mockito.verify(timerCallback).onTimer(watermarkTime1); + Mockito.verify(timerCallback).onTimer(watermarkTime2); Mockito.verifyNoMoreInteractions(timerCallback); underTest.advanceProcessingTime(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime1, processingTime2))); + Mockito.verify(timerCallback).onTimer(processingTime1); + Mockito.verify(timerCallback).onTimer(processingTime2); Mockito.verifyNoMoreInteractions(timerCallback); } @@ -142,9 +107,10 @@ public void testDeduplicate() throws Exception { underTest.setTimer(processingTime); underTest.setTimer(processingTime); underTest.advanceProcessingTime(timerCallback, new Instant(20)); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(processingTime))); underTest.advanceInputWatermark(timerCallback, new Instant(20)); - Mockito.verify(timerCallback).onTimers(argThat(timersAre(watermarkTime))); + + Mockito.verify(timerCallback).onTimer(processingTime); + Mockito.verify(timerCallback).onTimer(watermarkTime); Mockito.verifyNoMoreInteractions(timerCallback); } } From 6ef9a288e281a423905c2cba520274d1c4e4747b Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 30 Nov 2016 14:30:14 -0800 Subject: [PATCH 046/279] Shutdown DynamicSplit Executor in Cleanup This ensures that the threads will be shut off when the pipeline shuts down, enabling a JVM with no more work to do to shut down as well. --- .../beam/runners/direct/BoundedReadEvaluatorFactory.java | 6 ++++-- .../runners/direct/BoundedReadEvaluatorFactoryTest.java | 6 ++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 65b622f0b382..8874a04be224 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -58,7 +58,7 @@ final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory { */ private static final long REQUIRED_DYNAMIC_SPLIT_ORIGINAL_SIZE = 0; private final EvaluationContext evaluationContext; - private final ExecutorService executor = Executors.newCachedThreadPool(); + @VisibleForTesting final ExecutorService executor = Executors.newCachedThreadPool(); private final long minimumDynamicSplitSize; @@ -87,7 +87,9 @@ private TransformEvaluator createEvaluator( } @Override - public void cleanup() {} + public void cleanup() { + executor.shutdown(); + } /** * A {@link BoundedReadEvaluator} produces elements from an underlying {@link BoundedSource}, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index dee95a7819d0..b1ff6890b19a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -335,6 +335,12 @@ public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception { assertThat(TestSource.readerClosed, is(true)); } + @Test + public void cleanupShutsDownExecutor() { + factory.cleanup(); + assertThat(factory.executor.isShutdown(), is(true)); + } + private static class TestSource extends OffsetBasedSource { private static boolean readerClosed; private final Coder coder; From 38f0b11cc9028cf347e3c96b6e6116e5a5a9972d Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 30 Nov 2016 14:28:51 -0800 Subject: [PATCH 047/279] Preserves compressed windows in PushbackSideInputDoFnRunner --- .../core/PushbackSideInputDoFnRunner.java | 20 +++++++++++++++---- .../core/PushbackSideInputDoFnRunnerTest.java | 18 ++++++++++------- 2 files changed, 27 insertions(+), 11 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index 8c169da96e5f..460154da1c83 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -74,17 +74,29 @@ public Iterable> processElementInReadyWindows(WindowedValu processElement(elem); return Collections.emptyList(); } - ImmutableList.Builder> pushedBack = ImmutableList.builder(); + ImmutableList.Builder readyWindowsBuilder = ImmutableList.builder(); + ImmutableList.Builder pushedBackWindowsBuilder = ImmutableList.builder(); for (WindowedValue windowElem : elem.explodeWindows()) { BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows()); if (isReady(mainInputWindow)) { - processElement(windowElem); + readyWindowsBuilder.add(mainInputWindow); } else { notReadyWindows.add(mainInputWindow); - pushedBack.add(windowElem); + pushedBackWindowsBuilder.add(mainInputWindow); } } - return pushedBack.build(); + ImmutableList readyWindows = readyWindowsBuilder.build(); + ImmutableList pushedBackWindows = pushedBackWindowsBuilder.build(); + if (!readyWindows.isEmpty()) { + processElement( + WindowedValue.of( + elem.getValue(), elem.getTimestamp(), readyWindows, elem.getPane())); + } + return pushedBackWindows.isEmpty() + ? ImmutableList.>of() + : ImmutableList.of( + WindowedValue.of( + elem.getValue(), elem.getTimestamp(), pushedBackWindows, elem.getPane())); } private boolean isReady(BoundedWindow mainInputWindow) { diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java index 59a7c92e7d21..f8f4604c5503 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.core; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; -import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; @@ -27,7 +27,6 @@ import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.List; - import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; @@ -131,7 +130,7 @@ public void processElementSideInputNotReadyMultipleWindows() { PaneInfo.ON_TIME_AND_ONLY_FIRING); Iterable> multiWindowPushback = runner.processElementInReadyWindows(multiWindow); - assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows())); + assertThat(multiWindowPushback, contains(multiWindow)); assertThat(underlying.inputElems, Matchers.>emptyIterable()); } @@ -162,9 +161,14 @@ public void processElementSideInputNotReadySomeWindows() { assertThat( multiWindowPushback, containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L)))); - assertThat(underlying.inputElems, - containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING), - WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING))); + assertThat( + underlying.inputElems, + containsInAnyOrder( + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of(littleWindow, bigWindow), + PaneInfo.NO_FIRING))); } @Test @@ -188,7 +192,7 @@ public void processElementSideInputReadyAllWindows() { runner.processElementInReadyWindows(multiWindow); assertThat(multiWindowPushback, emptyIterable()); assertThat(underlying.inputElems, - containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray())); + containsInAnyOrder(ImmutableList.of(multiWindow).toArray())); } @Test From 265c79241f802f4d895648c1b1c4b75e6846d245 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 30 Nov 2016 11:10:20 -0800 Subject: [PATCH 048/279] Update examples archetype with runner profiles This makes it possible to run the examples on all runners. --- .../resources/archetype-resources/pom.xml | 123 +++++++++++++++--- 1 file changed, 103 insertions(+), 20 deletions(-) diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 031ee88aaf9a..df2e9f351268 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -26,6 +26,10 @@ jar + + 0.4.0-incubating-SNAPSHOT + + apache.snapshots @@ -85,36 +89,108 @@ - - - - org.apache.beam - beam-sdks-java-core - 0.4.0-incubating-SNAPSHOT - + + + direct-runner + + + + org.apache.beam + beam-runners-direct-java + ${beam.version} + runtime + + + - - - org.apache.beam - beam-runners-direct-java - 0.4.0-incubating-SNAPSHOT - runtime - + + apex-runner + + + + org.apache.beam + beam-runners-apex + ${beam.version} + runtime + + + + + + dataflow-runner + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${beam.version} + runtime + + + + + flink-runner + + + + org.apache.beam + beam-runners-flink_2.10 + ${beam.version} + runtime + + + + + + spark-runner + + + + org.apache.beam + beam-runners-spark + ${beam.version} + runtime + + + org.apache.spark + spark-streaming_2.10 + 1.6.2 + runtime + + + org.slf4j + jul-to-slf4j + + + + + com.fasterxml.jackson.module + jackson-module-scala_2.10 + 2.7.2 + runtime + + + + + + + org.apache.beam - beam-runners-google-cloud-dataflow-java - 0.4.0-incubating-SNAPSHOT - runtime + beam-sdks-java-core + ${beam.version} - + org.apache.beam beam-sdks-java-io-google-cloud-platform - 0.4.0-incubating-SNAPSHOT + ${beam.version} + com.google.api-client google-api-client @@ -129,7 +205,6 @@ - com.google.apis google-api-services-bigquery @@ -212,5 +287,13 @@ junit 4.11 + + + + org.apache.beam + beam-runners-direct-java + ${beam.version} + test + From d99829dd99db4090ceb7e5eefce50ee513c5458e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 17 Nov 2016 12:38:00 +0100 Subject: [PATCH 049/279] [BEAM-918] Allow users to define the storage level via pipeline options --- .../runners/spark/SparkPipelineOptions.java | 5 ++ .../spark/translation/BoundedDataset.java | 5 +- .../runners/spark/translation/Dataset.java | 2 +- .../spark/translation/EvaluationContext.java | 10 +++- .../translation/StorageLevelPTransform.java | 43 ++++++++++++++ .../translation/TransformTranslator.java | 27 +++++++++ .../streaming/UnboundedDataset.java | 13 ++++- .../spark/translation/StorageLevelTest.java | 56 +++++++++++++++++++ 8 files changed, 155 insertions(+), 6 deletions(-) create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java index 0fd790eba31b..3f8b3791f151 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java @@ -44,6 +44,11 @@ public interface SparkPipelineOptions Long getBatchIntervalMillis(); void setBatchIntervalMillis(Long batchInterval); + @Description("Batch default storage level") + @Default.String("MEMORY_ONLY") + String getStorageLevel(); + void setStorageLevel(String storageLevel); + @Description("Minimum time to spend on read, for each micro-batch.") @Default.Long(200) Long getMinReadTimeMillis(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index 774efb9916cd..1cfb0e02ed1a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -32,6 +32,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.storage.StorageLevel; /** * Holds an RDD or values for deferred conversion to an RDD if needed. PCollections are sometimes @@ -97,8 +98,8 @@ public WindowedValue apply(byte[] bytes) { } @Override - public void cache() { - rdd.cache(); + public void cache(String storageLevel) { + rdd.persist(StorageLevel.fromString(storageLevel)); } @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/Dataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/Dataset.java index 36b03feb77c7..b5d550e9d72d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/Dataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/Dataset.java @@ -26,7 +26,7 @@ */ public interface Dataset extends Serializable { - void cache(); + void cache(String storageLevel); void action(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 1183fbb73fc3..ae456098dc22 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; +import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.translation.streaming.UnboundedDataset; import org.apache.beam.sdk.AggregatorRetrievalException; @@ -155,7 +156,7 @@ public Dataset borrowDataset(PValue pvalue) { leaves.remove(dataset); if (multiReads.contains(pvalue)) { // Ensure the RDD is marked as cached - dataset.cache(); + dataset.cache(storageLevel()); } else { multiReads.add(pvalue); } @@ -172,7 +173,8 @@ Iterable> getPCollectionView(PCollectionView v */ public void computeOutputs() { for (Dataset dataset : leaves) { - dataset.cache(); // cache so that any subsequent get() is cheap. + // cache so that any subsequent get() is cheap. + dataset.cache(storageLevel()); dataset.action(); // force computation. } } @@ -295,4 +297,8 @@ public State waitUntilFinish(Duration duration) { private boolean isStreamingPipeline() { return jssc != null; } + + private String storageLevel() { + return runtime.getPipelineOptions().as(SparkPipelineOptions.class).getStorageLevel(); + } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java new file mode 100644 index 000000000000..6944dbf0a634 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.spark.translation; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollection; + +/** + * Get RDD storage level for the input PCollection (mostly used for testing purpose). + */ +public final class StorageLevelPTransform extends PTransform, PCollection> { + + @Override + public PCollection apply(PCollection input) { + return PCollection.createPrimitiveOutputInternal(input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED); + } + + @Override + public Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 60d668e3b1bd..66da18131614 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -26,6 +26,7 @@ import com.google.common.collect.Maps; import java.io.IOException; +import java.util.Collections; import java.util.Map; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; @@ -34,6 +35,7 @@ import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.SourceRDD; import org.apache.beam.runners.spark.io.hadoop.HadoopIO; import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper; @@ -42,6 +44,7 @@ import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.TextIO; @@ -78,6 +81,7 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; + import scala.Tuple2; @@ -583,6 +587,27 @@ public void evaluate(View.CreatePCollectionView transform, }; } + private static TransformEvaluator storageLevel() { + return new TransformEvaluator() { + @Override + public void evaluate(StorageLevelPTransform transform, EvaluationContext context) { + JavaRDD rdd = ((BoundedDataset) (context).borrowDataset(transform)).getRDD(); + JavaSparkContext javaSparkContext = context.getSparkContext(); + + WindowedValue.ValueOnlyWindowedValueCoder windowCoder = + WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + JavaRDD output = + javaSparkContext.parallelize( + CoderHelpers.toByteArrays( + Collections.singletonList(rdd.getStorageLevel().description()), + StringUtf8Coder.of())) + .map(CoderHelpers.fromByteFunction(windowCoder)); + + context.putDataset(transform, new BoundedDataset(output)); + } + }; + } + private static final Map, TransformEvaluator> EVALUATORS = Maps .newHashMap(); @@ -602,6 +627,8 @@ public void evaluate(View.CreatePCollectionView transform, EVALUATORS.put(View.AsIterable.class, viewAsIter()); EVALUATORS.put(View.CreatePCollectionView.class, createPCollView()); EVALUATORS.put(Window.Bound.class, window()); + // mostly test evaluators + EVALUATORS.put(StorageLevelPTransform.class, storageLevel()); } /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java index 67adee2b18aa..d059c7e3dec3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java @@ -31,12 +31,17 @@ import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * DStream holder Can also crate a DStream from a supplied queue of values, but mainly for testing. */ public class UnboundedDataset implements Dataset { + + private static final Logger LOG = LoggerFactory.getLogger(UnboundedDataset.class); + // only set if creating a DStream from a static collection @Nullable private transient JavaStreamingContext jssc; @@ -81,11 +86,17 @@ JavaDStream> getDStream() { return dStream; } - @Override public void cache() { dStream.cache(); } + @Override + public void cache(String storageLevel) { + // we "force" MEMORY storage level in streaming + LOG.warn("Provided StorageLevel ignored for stream, using default level"); + cache(); + } + @Override public void action() { dStream.foreachRDD(new VoidFunction>>() { diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java new file mode 100644 index 000000000000..48105e1871f4 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.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.spark.translation; + +import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptions; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; + +/** + * Test the RDD storage level defined by user. + */ +public class StorageLevelTest { + + @Rule + public final transient SparkTestPipelineOptions pipelineOptions = new SparkTestPipelineOptions(); + + @Test + public void test() throws Exception { + pipelineOptions.getOptions().setStorageLevel("DISK_ONLY"); + Pipeline p = Pipeline.create(pipelineOptions.getOptions()); + + PCollection pCollection = p.apply(Create.of("foo")); + + // by default, the Spark runner doesn't cache the RDD if it accessed only one time. + // So, to "force" the caching of the RDD, we have to call the RDD at least two time. + // That's why we are using Count fn on the PCollection. + pCollection.apply(Count.globally()); + + PCollection output = pCollection.apply(new StorageLevelPTransform()); + + PAssert.thatSingleton(output).isEqualTo("Disk Serialized 1x Replicated"); + + p.run(); + } + +} From ab1f1ad012bc559cdb099319a516e4437eed2825 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 29 Nov 2016 14:29:47 -0800 Subject: [PATCH 050/279] Move TransformHierarchy Maintenance into it This reduces the complexity of Pipeline.applyInternal by keeping the responsiblities to passing a node into the Transform Hierarchy, enforcing name uniqueness, and causing the runner to expand the PTransform. This logic is moved to the appropriate application sites. --- .../direct/KeyedPValueTrackingVisitor.java | 2 +- .../DataflowPipelineTranslatorTest.java | 2 +- .../java/org/apache/beam/sdk/Pipeline.java | 117 +++--------- .../beam/sdk/runners/TransformHierarchy.java | 126 ++++++++---- .../beam/sdk/runners/TransformTreeNode.java | 165 +++++++++------- .../sdk/runners/TransformHierarchyTest.java | 180 +++++++++++++----- .../beam/sdk/runners/TransformTreeTest.java | 4 +- 7 files changed, 340 insertions(+), 256 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 7c4376a2c4b4..47b08576546a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -74,7 +74,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { if (node.isRootNode()) { finalized = true; } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) { - keyedValues.addAll(node.getExpandedOutputs()); + keyedValues.addAll(node.getOutput().expand()); } } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index c925454a2c3f..95c7132c88d6 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -669,7 +669,7 @@ public void testPartiallyBoundFailure() throws IOException { PCollection input = p.begin() .apply(Create.of(1, 2, 3)); - thrown.expect(IllegalStateException.class); + thrown.expect(IllegalArgumentException.class); input.apply(new PartiallyBoundOutputCreator()); Assert.fail("Failure expected from use of partially bound output"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index 9edf4962c97d..c8a4439ba899 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -17,10 +17,11 @@ */ package org.apache.beam.sdk; +import static com.google.common.base.Preconditions.checkState; + import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import org.apache.beam.sdk.coders.CoderRegistry; @@ -31,7 +32,6 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.UserCodeException; @@ -282,14 +282,12 @@ public void visitValue(PValue value, TransformTreeNode producer) { } *

    Typically invoked by {@link PipelineRunner} subclasses. */ public void traverseTopologically(PipelineVisitor visitor) { - Set visitedValues = new HashSet<>(); - // Visit all the transforms, which should implicitly visit all the values. - transforms.visit(visitor, visitedValues); - if (!visitedValues.containsAll(values)) { - throw new RuntimeException( - "internal error: should have visited all the values " - + "after visiting all the transforms"); - } + Set visitedValues = + // Visit all the transforms, which should implicitly visit all the values. + transforms.visit(visitor); + checkState( + visitedValues.containsAll(values), + "internal error: should have visited all the values after visiting all the transforms"); } /** @@ -351,116 +349,49 @@ public String toString() { * * @see Pipeline#apply */ - private - OutputT applyInternal(String name, InputT input, - PTransform transform) { - input.finishSpecifying(); + private OutputT applyInternal( + String name, InputT input, PTransform transform) { + String namePrefix = transforms.getCurrent().getFullName(); + String uniqueName = uniquifyInternal(namePrefix, name); - TransformTreeNode parent = transforms.getCurrent(); - String namePrefix = parent.getFullName(); - String fullName = uniquifyInternal(namePrefix, name); - - boolean nameIsUnique = fullName.equals(buildName(namePrefix, name)); + boolean nameIsUnique = uniqueName.equals(buildName(namePrefix, name)); if (!nameIsUnique) { switch (getOptions().getStableUniqueNames()) { case OFF: break; case WARNING: - LOG.warn("Transform {} does not have a stable unique name. " - + "This will prevent updating of pipelines.", fullName); + LOG.warn( + "Transform {} does not have a stable unique name. " + + "This will prevent updating of pipelines.", + uniqueName); break; case ERROR: throw new IllegalStateException( - "Transform " + fullName + " does not have a stable unique name. " - + "This will prevent updating of pipelines."); + "Transform " + + uniqueName + + " does not have a stable unique name. " + + "This will prevent updating of pipelines."); default: throw new IllegalArgumentException( "Unrecognized value for stable unique names: " + getOptions().getStableUniqueNames()); } } - TransformTreeNode child = - new TransformTreeNode(parent, transform, fullName, input); - parent.addComposite(child); - - transforms.addInput(child, input); - LOG.debug("Adding {} to {}", transform, this); + transforms.pushNode(uniqueName, input, transform); try { - transforms.pushNode(child); + transforms.finishSpecifyingInput(); transform.validate(input); OutputT output = runner.apply(transform, input); - transforms.setOutput(child, output); + transforms.setOutput(output); - AppliedPTransform applied = AppliedPTransform.of( - child.getFullName(), input, output, transform); - // recordAsOutput is a NOOP if already called; - output.recordAsOutput(applied); - verifyOutputState(output, child); return output; } finally { transforms.popNode(); } } - /** - * Returns all producing transforms for the {@link PValue PValues} contained - * in {@code output}. - */ - private List> getProducingTransforms(POutput output) { - List> producingTransforms = new ArrayList<>(); - for (PValue value : output.expand()) { - AppliedPTransform transform = value.getProducingTransformInternal(); - if (transform != null) { - producingTransforms.add(transform); - } - } - return producingTransforms; - } - - /** - * Verifies that the output of a {@link PTransform} is correctly configured in its - * {@link TransformTreeNode} in the {@link Pipeline} graph. - * - *

    A non-composite {@link PTransform} must have all - * of its outputs registered as produced by that {@link PTransform}. - * - *

    A composite {@link PTransform} must have all of its outputs - * registered as produced by the contained primitive {@link PTransform PTransforms}. - * They have each had the above check performed already, when - * they were applied, so the only possible failure state is - * that the composite {@link PTransform} has returned a primitive output. - */ - private void verifyOutputState(POutput output, TransformTreeNode node) { - if (!node.isCompositeNode()) { - PTransform thisTransform = node.getTransform(); - List> producingTransforms = getProducingTransforms(output); - for (AppliedPTransform producingTransform : producingTransforms) { - // Using != because object identity indicates that the transforms - // are the same node in the pipeline - if (thisTransform != producingTransform.getTransform()) { - throw new IllegalArgumentException("Output of non-composite transform " - + thisTransform + " is registered as being produced by" - + " a different transform: " + producingTransform); - } - } - } else { - PTransform thisTransform = node.getTransform(); - List> producingTransforms = getProducingTransforms(output); - for (AppliedPTransform producingTransform : producingTransforms) { - // Using == because object identity indicates that the transforms - // are the same node in the pipeline - if (thisTransform == producingTransform.getTransform()) { - throw new IllegalStateException("Output of composite transform " - + thisTransform + " is registered as being produced by it," - + " but the output of every composite transform should be" - + " produced by a primitive transform contained therein."); - } - } - } - } - /** * Returns the configured {@link PipelineRunner}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 0a4bb08a315d..d3fd4974bbb9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -17,14 +17,17 @@ */ package org.apache.beam.sdk.runners; +import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import java.util.Deque; +import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedList; +import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; @@ -34,70 +37,109 @@ * associated {@link PValue}s. */ public class TransformHierarchy { - private final Deque transformStack = new LinkedList<>(); - private final Map producingTransformNode = new HashMap<>(); + private final TransformTreeNode root; + private final Map producers; + // Maintain a stack based on the enclosing nodes + private TransformTreeNode current; - /** - * Create a {@code TransformHierarchy} containing a root node. - */ public TransformHierarchy() { - // First element in the stack is the root node, holding all child nodes. - transformStack.add(new TransformTreeNode(null, null, "", null)); + root = TransformTreeNode.root(this); + current = root; + producers = new HashMap<>(); } /** - * Returns the last TransformTreeNode on the stack. + * Adds the named {@link PTransform} consuming the provided {@link PInput} as a node in this + * {@link TransformHierarchy} as a child of the current node, and sets it to be the current node. + * + *

    This call should be finished by expanding and recursively calling {@link #pushNode(String, + * PInput, PTransform)}, calling {@link #finishSpecifyingInput()}, setting the output with {@link + * #setOutput(POutput)}, and ending with a call to {@link #popNode()}. + * + * @return the added node */ - public TransformTreeNode getCurrent() { - return transformStack.peek(); + public TransformTreeNode pushNode(String name, PInput input, PTransform transform) { + checkNotNull( + transform, "A %s must be provided for all Nodes", PTransform.class.getSimpleName()); + checkNotNull( + name, "A name must be provided for all %s Nodes", PTransform.class.getSimpleName()); + checkNotNull( + input, "An input must be provided for all %s Nodes", PTransform.class.getSimpleName()); + current = TransformTreeNode.subtransform(current, transform, name, input); + return current; } /** - * Add a TransformTreeNode to the stack. + * Finish specifying all of the input {@link PValue PValues} of the current {@link + * TransformTreeNode}. Ensures that all of the inputs to the current node have been fully + * specified, and have been produced by a node in this graph. */ - public void pushNode(TransformTreeNode current) { - transformStack.push(current); + public void finishSpecifyingInput() { + // Inputs must be completely specified before they are consumed by a transform. + current.getInput().finishSpecifying(); + for (PValue inputValue : current.getInput().expand()) { + checkState(producers.get(inputValue) != null, "Producer unknown for input %s", inputValue); + inputValue.finishSpecifying(); + } } /** - * Removes the last TransformTreeNode from the stack. + * Set the output of the current {@link TransformTreeNode}. If the output is new (setOutput has + * not previously been called with it as the parameter), the current node is set as the producer + * of that {@link POutput}. + * + *

    Also validates the output - specifically, a Primitive {@link PTransform} produces all of + * its outputs, and a Composite {@link PTransform} produces none of its outputs. Verifies that the + * expanded output does not contain {@link PValue PValues} produced by both this node and other + * nodes. */ - public void popNode() { - transformStack.pop(); - checkState(!transformStack.isEmpty()); + public void setOutput(POutput output) { + for (PValue value : output.expand()) { + if (!producers.containsKey(value)) { + producers.put(value, current); + } + } + current.setOutput(output); + // TODO: Replace with a "generateDefaultNames" method. + output.recordAsOutput(current.toAppliedPTransform()); } /** - * Adds an input to the given node. - * - *

    This forces the producing node to be finished. + * Pops the current node off the top of the stack, finishing it. Outputs of the node are finished + * once they are consumed as input. */ - public void addInput(TransformTreeNode node, PInput input) { - for (PValue i : input.expand()) { - TransformTreeNode producer = producingTransformNode.get(i); - checkState(producer != null, "Producer unknown for input: %s", i); + public void popNode() { + current.finishSpecifying(); + current = current.getEnclosingNode(); + checkState(current != null, "Can't pop the root node of a TransformHierarchy"); + } - producer.finishSpecifying(); - node.addInputProducer(i, producer); - } + TransformTreeNode getProducer(PValue produced) { + return producers.get(produced); } /** - * Sets the output of a transform node. + * Returns all producing transforms for the {@link PValue PValues} contained + * in {@code output}. */ - public void setOutput(TransformTreeNode producer, POutput output) { - producer.setOutput(output); - - for (PValue o : output.expand()) { - producingTransformNode.put(o, producer); + List getProducingTransforms(POutput output) { + List producingTransforms = new ArrayList<>(); + for (PValue value : output.expand()) { + TransformTreeNode producer = getProducer(value); + if (producer != null) { + producingTransforms.add(producer); + } } + return producingTransforms; } - /** - * Visits all nodes in the transform hierarchy, in transitive order. - */ - public void visit(Pipeline.PipelineVisitor visitor, - Set visitedNodes) { - transformStack.peekFirst().visit(visitor, visitedNodes); + public Set visit(PipelineVisitor visitor) { + Set visitedValues = new HashSet<>(); + root.visit(visitor, visitedValues); + return visitedValues; + } + + public TransformTreeNode getCurrent() { + return current; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java index d16b8281dc66..ea94bd958d5d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java @@ -17,18 +17,19 @@ */ package org.apache.beam.sdk.runners; -import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; +import java.util.HashSet; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; +import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; @@ -39,6 +40,7 @@ * for initialization and ordered visitation. */ public class TransformTreeNode { + private final TransformHierarchy hierarchy; private final TransformTreeNode enclosingNode; // The PTransform for this node, which may be a composite PTransform. @@ -51,10 +53,6 @@ public class TransformTreeNode { // Nodes for sub-transforms of a composite transform. private final Collection parts = new ArrayList<>(); - // Inputs to the transform, in expanded form and mapped to the producer - // of the input. - private final Map inputs = new HashMap<>(); - // Input to the transform, in unexpanded form. private final PInput input; @@ -62,28 +60,57 @@ public class TransformTreeNode { // Output of the transform, in unexpanded form. private POutput output; - private boolean finishedSpecifying = false; + @VisibleForTesting + boolean finishedSpecifying = false; + + /** + * Create a root {@link TransformTreeNode}. This transform is the root of the provided {@link + * TransformHierarchy} - it has no enclosing node, no {@link PTransform}, no {@link PInput input}, + * no {@link POutput output}, and an empty name. It contains all {@link PTransform transforms} + * within a {@link Pipeline} as component transforms. + */ + public static TransformTreeNode root(TransformHierarchy hierarchy) { + return new TransformTreeNode(hierarchy, null, null, "", null); + } + + /** + * Create a subtransform of the provided {@link TransformTreeNode node}. The enclosing node is a + * composite that contains this transform. + * + *

    The returned node is a component node of the enclosing node. + */ + public static TransformTreeNode subtransform( + TransformTreeNode enclosing, PTransform transform, String fullName, PInput input) { + checkNotNull(enclosing); + checkNotNull(transform); + checkNotNull(fullName); + checkNotNull(input); + TransformTreeNode node = + new TransformTreeNode(enclosing.hierarchy, enclosing, transform, fullName, input); + enclosing.addComposite(node); + return node; + } /** * Creates a new TransformTreeNode with the given parent and transform. * - *

    EnclosingNode and transform may both be null for - * a root-level node, which holds all other nodes. + *

    EnclosingNode and transform may both be null for a root-level node, which holds all other + * nodes. * * @param enclosingNode the composite node containing this node * @param transform the PTransform tracked by this node * @param fullName the fully qualified name of the transform * @param input the unexpanded input to the transform */ - public TransformTreeNode(@Nullable TransformTreeNode enclosingNode, - @Nullable PTransform transform, - String fullName, - @Nullable PInput input) { + private TransformTreeNode( + TransformHierarchy hierarchy, + @Nullable TransformTreeNode enclosingNode, + @Nullable PTransform transform, + String fullName, + @Nullable PInput input) { + this.hierarchy = hierarchy; this.enclosingNode = enclosingNode; this.transform = transform; - checkArgument((enclosingNode == null && transform == null) - || (enclosingNode != null && transform != null), - "EnclosingNode and transform must both be specified, or both be null"); this.fullName = fullName; this.input = input; } @@ -113,21 +140,23 @@ public void addComposite(TransformTreeNode node) { } /** - * Returns true if this node represents a composite transform that does not perform - * processing of its own, but merely encapsulates a sub-pipeline (which may be empty). + * Returns true if this node represents a composite transform that does not perform processing of + * its own, but merely encapsulates a sub-pipeline (which may be empty). * - *

    Note that a node may be composite with no sub-transforms if it returns its input directly + *

    Note that a node may be composite with no sub-transforms if it returns its input directly * extracts a component of a tuple, or other operations that occur at pipeline assembly time. */ public boolean isCompositeNode() { - return !parts.isEmpty() || returnsOthersOutput() || isRootNode(); + return !parts.isEmpty() || isRootNode() || returnsOthersOutput(); } private boolean returnsOthersOutput() { PTransform transform = getTransform(); - for (PValue output : getExpandedOutputs()) { - if (!output.getProducingTransformInternal().getTransform().equals(transform)) { - return true; + if (output != null) { + for (PValue outputValue : output.expand()) { + if (!hierarchy.getProducer(outputValue).getTransform().equals(transform)) { + return true; + } } } return false; @@ -141,14 +170,6 @@ public String getFullName() { return fullName; } - /** - * Adds an input to the transform node. - */ - public void addInputProducer(PValue expandedInput, TransformTreeNode producer) { - checkState(!finishedSpecifying); - inputs.put(expandedInput, producer); - } - /** * Returns the transform input, in unexpanded form. */ @@ -156,21 +177,38 @@ public PInput getInput() { return input; } - /** - * Returns a mapping of inputs to the producing nodes for all inputs to - * the transform. - */ - public Map getInputs() { - return Collections.unmodifiableMap(inputs); - } - /** * Adds an output to the transform node. */ public void setOutput(POutput output) { checkState(!finishedSpecifying); - checkState(this.output == null); + checkState(this.output == null, "Tried to specify more than one output for %s", getFullName()); + checkNotNull(output, "Tried to set the output of %s to null", getFullName()); this.output = output; + + // Validate that a primitive transform produces only primitive output, and a composite transform + // does not produce primitive output. + Set outputProducers = new HashSet<>(); + for (PValue outputValue : output.expand()) { + outputProducers.add(hierarchy.getProducer(outputValue)); + } + if (outputProducers.contains(this) && outputProducers.size() != 1) { + Set otherProducerNames = new HashSet<>(); + for (TransformTreeNode outputProducer : outputProducers) { + if (outputProducer != this) { + otherProducerNames.add(outputProducer.getFullName()); + } + } + throw new IllegalArgumentException( + String.format( + "Output of transform [%s] contains a %s produced by it as well as other Transforms. " + + "A primitive transform must produce all of its outputs, and outputs of a " + + "composite transform must be produced by a component transform or be part of" + + "the input." + + "%n Other Outputs: %s" + + "%n Other Producers: %s", + getFullName(), POutput.class.getSimpleName(), output.expand(), otherProducerNames)); + } } /** @@ -180,17 +218,10 @@ public POutput getOutput() { return output; } - /** - * Returns the transform outputs, in expanded form. - */ - public Collection getExpandedOutputs() { - if (output != null) { - return output.expand(); - } else { - return Collections.emptyList(); - } + AppliedPTransform toAppliedPTransform() { + return AppliedPTransform.of( + getFullName(), getInput(), getOutput(), (PTransform) getTransform()); } - /** * Visit the transform node. * @@ -204,10 +235,12 @@ public void visit(PipelineVisitor visitor, finishSpecifying(); } - // Visit inputs. - for (Map.Entry entry : inputs.entrySet()) { - if (visitedValues.add(entry.getKey())) { - visitor.visitValue(entry.getKey(), entry.getValue()); + if (!isRootNode()) { + // Visit inputs. + for (PValue inputValue : input.expand()) { + if (visitedValues.add(inputValue)) { + visitor.visitValue(inputValue, hierarchy.getProducer(inputValue)); + } } } @@ -224,10 +257,12 @@ public void visit(PipelineVisitor visitor, visitor.visitPrimitiveTransform(this); } - // Visit outputs. - for (PValue pValue : getExpandedOutputs()) { - if (visitedValues.add(pValue)) { - visitor.visitValue(pValue, this); + if (!isRootNode()) { + // Visit outputs. + for (PValue pValue : output.expand()) { + if (visitedValues.add(pValue)) { + visitor.visitValue(pValue, this); + } } } } @@ -243,15 +278,5 @@ public void finishSpecifying() { return; } finishedSpecifying = true; - - for (TransformTreeNode input : inputs.values()) { - if (input != null) { - input.finishSpecifying(); - } - } - - if (output != null) { - output.finishSpecifyingOutput(); - } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index c28f23efd607..3bf6d6440122 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; import java.util.HashSet; @@ -30,9 +31,15 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.hamcrest.Matchers; import org.junit.Before; @@ -63,7 +70,7 @@ public void getCurrentNoPushReturnsRoot() { } @Test - public void popWithoutPushThrows() { + public void pushWithoutPushFails() { thrown.expect(IllegalStateException.class); hierarchy.popNode(); } @@ -71,72 +78,153 @@ public void popWithoutPushThrows() { @Test public void pushThenPopSucceeds() { TransformTreeNode root = hierarchy.getCurrent(); - TransformTreeNode node = - new TransformTreeNode(hierarchy.getCurrent(), Create.of(1), "Create", PBegin.in(pipeline)); - hierarchy.pushNode(node); + TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); assertThat(hierarchy.getCurrent(), equalTo(node)); hierarchy.popNode(); + assertThat(node.finishedSpecifying, is(true)); assertThat(hierarchy.getCurrent(), equalTo(root)); } + @Test + public void emptyCompositeSucceeds() { + PCollection created = + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); + hierarchy.setOutput(created); + hierarchy.popNode(); + PCollectionList pcList = PCollectionList.of(created); + + TransformTreeNode emptyTransform = + hierarchy.pushNode( + "Extract", + pcList, + new PTransform, PCollection>() { + @Override + public PCollection apply(PCollectionList input) { + return input.get(0); + } + }); + hierarchy.setOutput(created); + hierarchy.popNode(); + assertThat(hierarchy.getProducer(created), equalTo(node)); + assertThat( + "A Transform that produces non-primtive output should be composite", + emptyTransform.isCompositeNode(), + is(true)); + } + + @Test + public void producingOwnAndOthersOutputsFails() { + PCollection created = + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); + hierarchy.setOutput(created); + hierarchy.popNode(); + PCollectionList pcList = PCollectionList.of(created); + + final PCollectionList appended = + pcList.and( + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)); + hierarchy.pushNode( + "AddPc", + pcList, + new PTransform, PCollectionList>() { + @Override + public PCollectionList apply(PCollectionList input) { + return appended; + } + }); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("produced by it as well as other Transforms"); + thrown.expectMessage("primitive transform must produce all of its outputs"); + thrown.expectMessage("composite transform must be produced by a component transform"); + thrown.expectMessage("AddPc"); + thrown.expectMessage("Create"); + thrown.expectMessage(appended.expand().toString()); + hierarchy.setOutput(appended); + } + @Test public void visitVisitsAllPushed() { TransformTreeNode root = hierarchy.getCurrent(); - Create.Values create = Create.of(1); - PCollection created = pipeline.apply(create); PBegin begin = PBegin.in(pipeline); - TransformTreeNode compositeNode = - new TransformTreeNode(root, create, "Create", begin); - root.addComposite(compositeNode); - TransformTreeNode primitiveNode = - new TransformTreeNode( - compositeNode, Read.from(CountingSource.upTo(1L)), "Create/Read", begin); - compositeNode.addComposite(primitiveNode); - - TransformTreeNode otherPrimitive = - new TransformTreeNode( - root, MapElements.via(new SimpleFunction() { - @Override - public Integer apply(Integer input) { - return input; - } - }), "ParDo", created); - root.addComposite(otherPrimitive); - otherPrimitive.addInputProducer(created, primitiveNode); + Create.Values create = Create.of(1L); + Read.Bounded read = Read.from(CountingSource.upTo(1L)); - hierarchy.pushNode(compositeNode); - hierarchy.pushNode(primitiveNode); + PCollection created = + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + + MapElements map = MapElements.via(new SimpleFunction() { + @Override + public Long apply(Long input) { + return input; + } + }); + + PCollection mapped = + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + + TransformTreeNode compositeNode = hierarchy.pushNode("Create", begin, create); + assertThat(hierarchy.getCurrent(), equalTo(compositeNode)); + assertThat(compositeNode.getInput(), Matchers.equalTo(begin)); + assertThat(compositeNode.getTransform(), Matchers.>equalTo(create)); + // Not yet set + assertThat(compositeNode.getOutput(), nullValue()); + assertThat(compositeNode.getEnclosingNode().isRootNode(), is(true)); + + TransformTreeNode primitiveNode = hierarchy.pushNode("Create/Read", begin, read); + assertThat(hierarchy.getCurrent(), equalTo(primitiveNode)); + hierarchy.setOutput(created); hierarchy.popNode(); + assertThat(primitiveNode.getOutput(), Matchers.equalTo(created)); + assertThat(primitiveNode.getInput(), Matchers.equalTo(begin)); + assertThat(primitiveNode.getTransform(), Matchers.>equalTo(read)); + assertThat(primitiveNode.getEnclosingNode(), equalTo(compositeNode)); + + hierarchy.setOutput(created); + // The composite is listed as outputting a PValue created by the contained primitive + assertThat(compositeNode.getOutput(), Matchers.equalTo(created)); + // The producer of that PValue is still the primitive in which it is first output + assertThat(hierarchy.getProducer(created), equalTo(primitiveNode)); hierarchy.popNode(); - hierarchy.pushNode(otherPrimitive); + + TransformTreeNode otherPrimitive = hierarchy.pushNode("ParDo", created, map); + hierarchy.setOutput(mapped); hierarchy.popNode(); final Set visitedCompositeNodes = new HashSet<>(); final Set visitedPrimitiveNodes = new HashSet<>(); final Set visitedValuesInVisitor = new HashSet<>(); - Set visitedValues = new HashSet<>(); - hierarchy.visit(new PipelineVisitor.Defaults() { - @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { - visitedCompositeNodes.add(node); - return CompositeBehavior.ENTER_TRANSFORM; - } - - @Override - public void visitPrimitiveTransform(TransformTreeNode node) { - visitedPrimitiveNodes.add(node); - } - - @Override - public void visitValue(PValue value, TransformTreeNode producer) { - visitedValuesInVisitor.add(value); - } - }, visitedValues); + Set visitedValues = + hierarchy.visit( + new PipelineVisitor.Defaults() { + @Override + public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + visitedCompositeNodes.add(node); + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void visitPrimitiveTransform(TransformTreeNode node) { + visitedPrimitiveNodes.add(node); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + visitedValuesInVisitor.add(value); + } + }); assertThat(visitedCompositeNodes, containsInAnyOrder(root, compositeNode)); assertThat(visitedPrimitiveNodes, containsInAnyOrder(primitiveNode, otherPrimitive)); - assertThat(visitedValuesInVisitor, Matchers.containsInAnyOrder(created)); + assertThat(visitedValuesInVisitor, Matchers.containsInAnyOrder(created, mapped)); + assertThat(visitedValuesInVisitor, equalTo(visitedValues)); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java index def3a027e9d7..b95fa70fe785 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import java.io.File; import java.util.Arrays; @@ -169,14 +168,13 @@ public void visitPrimitiveTransform(TransformTreeNode node) { assertTrue(left.equals(EnumSet.of(TransformsSeen.SAMPLE_ANY))); } - @Test(expected = IllegalStateException.class) + @Test(expected = IllegalArgumentException.class) public void testOutputChecking() throws Exception { Pipeline p = TestPipeline.create(); p.apply(new InvalidCompositeTransform()); p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() {}); - fail("traversal should have failed with an IllegalStateException"); } @Test From 1094fa6ac32046b4c092294b3cee046c91aea5a1 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 1 Dec 2016 09:15:28 -0800 Subject: [PATCH 051/279] Add a test of ReleaseInfo --- .../apache/beam/sdk/util/ReleaseInfoTest.java | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java new file mode 100644 index 000000000000..fabb7e293c2e --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +/** + * Tests for {@link ReleaseInfo}. + */ +public class ReleaseInfoTest { + + @Test + public void getReleaseInfo() throws Exception { + ReleaseInfo info = ReleaseInfo.getReleaseInfo(); + + // Validate name + assertThat(info.getName(), containsString("Beam")); + + // Validate semantic version + String version = info.getVersion(); + String pattern = "\\d+\\.\\d+\\.\\d+.*"; + assertTrue( + String.format("%s does not match pattern %s", version, pattern), + version.matches(pattern)); + } +} From b36048bd0e558fea281a1ec42aa8435db09dbe64 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 1 Dec 2016 10:22:15 -0800 Subject: [PATCH 052/279] Revert "Move resource filtering later to avoid spurious rebuilds" This reverts commit 2422365719c71cade97e1e74f1fb7f42b264244f. --- sdks/java/core/pom.xml | 29 +++++++---------------------- 1 file changed, 7 insertions(+), 22 deletions(-) diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index f842be71d108..ad848469c413 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -40,6 +40,13 @@ + + + src/main/resources + true + + + @@ -72,28 +79,6 @@ - - org.apache.maven.plugins - maven-resources-plugin - - - resources - compile - - resources - - - - - src/main/resources - true - - - - - - - org.apache.maven.plugins maven-jar-plugin From 87ff5ac36bb9cc62fa4864ffa7b5a5e495b9a4a1 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 26 Oct 2016 16:05:01 -0700 Subject: [PATCH 053/279] Improve Splittable DoFn Makes Splittable DoFn be more like a real DoFn: - Adds support for side inputs and outputs to SDF - Teaches `ProcessFn` to work with exploded windows inside the `KeyedWorkItem`. It works with them by un-exploding the windows in the `Iterable>` into a single `WindowedValue`, since the values and timestamps are guaranteed to be the same. Makes SplittableParDo.ProcessFn not use the (now unavailable) OldDoFn state and timers API: - Makes `ProcessFn` be a primitive transform with its own `ParDoEvaluator`. As a nice side effect, this enables the runner to provide additional hooks into it - e.g. for giving the runner access to the restriction tracker (in later PRs) - For consistency, moves declaration of `GBKIntoKeyedWorkItems` primitive transform into `SplittableParDo`, alongside the `SplittableProcessElements` transform - Preserves compressed representation of `WindowedValue`'s in `PushbackSideInputDoFnRunner` - Uses OutputWindowedValue in SplittableParDo.ProcessFn Proper lifecycle management for wrapped fn. - Caches underlying fn using DoFnLifecycleManager, so its @Setup and @Teardown methods are called. - Calls @StartBundle and @FinishBundle methods on the underlying fn explicitly. Output from them is prohibited, since an SDF is only allowed to output after a successful RestrictionTracker.tryClaim. It's possible that an SDF should not be allowed to have StartBundle/FinishBundle methods at all, but I'm not sure. --- .../core/ElementAndRestrictionCoder.java | 8 + .../runners/core/GBKIntoKeyedWorkItems.java | 55 --- .../beam/runners/core/SplittableParDo.java | 378 ++++++++++++++---- .../runners/core/SplittableParDoTest.java | 134 +++++-- ...tGBKIntoKeyedWorkItemsOverrideFactory.java | 41 +- .../beam/runners/direct/DirectGroupByKey.java | 2 +- .../beam/runners/direct/DirectRunner.java | 8 +- .../runners/direct/DoFnLifecycleManager.java | 4 +- .../beam/runners/direct/ParDoEvaluator.java | 26 +- .../runners/direct/ParDoEvaluatorFactory.java | 63 ++- .../direct/ParDoMultiOverrideFactory.java | 2 +- ...ttableProcessElementsEvaluatorFactory.java | 144 +++++++ .../direct/TransformEvaluatorRegistry.java | 5 + .../runners/direct/SplittableDoFnTest.java | 194 ++++++++- .../org/apache/beam/sdk/transforms/DoFn.java | 12 + .../beam/sdk/transforms/DoFnTester.java | 51 ++- .../sdk/util/state/TimerInternalsFactory.java | 36 ++ 17 files changed, 905 insertions(+), 258 deletions(-) delete mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java index 6dec8e2e2a6e..64c1e14156c0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java @@ -64,4 +64,12 @@ public ElementAndRestriction decode(InputStream inStream RestrictionT value = restrictionCoder.decode(inStream, context); return ElementAndRestriction.of(key, value); } + + public Coder getElementCoder() { + return elementCoder; + } + + public Coder getRestrictionCoder() { + return restrictionCoder; + } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java deleted file mode 100644 index 304e3495e18b..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java +++ /dev/null @@ -1,55 +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.core; - -import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItemCoder; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - -/** - * Interface for creating a runner-specific {@link GroupByKey GroupByKey-like} {@link PTransform} - * that produces {@link KeyedWorkItem KeyedWorkItems} so that downstream transforms can access state - * and timers. - */ -@Experimental(Experimental.Kind.SPLITTABLE_DO_FN) -public class GBKIntoKeyedWorkItems - extends PTransform>, PCollection>> { - @Override - public PCollection> apply(PCollection> input) { - checkArgument(input.getCoder() instanceof KvCoder, - "Expected input coder to be KvCoder, but was %s", - input.getCoder().getClass().getSimpleName()); - - KvCoder kvCoder = (KvCoder) input.getCoder(); - Coder> coder = KeyedWorkItemCoder.of( - kvCoder.getKeyCoder(), kvCoder.getValueCoder(), - input.getWindowingStrategy().getWindowFn().windowCoder()); - PCollection> collection = PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); - collection.setCoder((Coder) coder); - return collection; - } -} diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index c38ab2f514e3..80fd17b4e748 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -19,17 +19,22 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import java.util.List; import java.util.UUID; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; 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.OldDoFn; +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.SerializableFunction; @@ -45,21 +50,30 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.KeyedWorkItemCoder; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; 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.State; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; +import org.apache.beam.sdk.util.state.TimerInternalsFactory; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.util.state.WatermarkHoldState; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypedPValue; import org.joda.time.Instant; /** @@ -80,31 +94,53 @@ * ParDo.of(splittable DoFn)}, but not for direct use by pipeline writers. */ @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) -public class SplittableParDo< - InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> - extends PTransform, PCollection> { - private final DoFn fn; - private final DoFnSignature signature; +public class SplittableParDo + extends PTransform, PCollectionTuple> { + private final ParDo.BoundMulti parDo; /** - * Creates the transform for the given original {@link ParDo} and {@link DoFn}. + * Creates the transform for the given original multi-output {@link ParDo}. * - * @param fn The splittable {@link DoFn} inside the original {@link ParDo} transform. + * @param parDo The splittable {@link ParDo} transform. */ - public SplittableParDo(DoFn fn) { - checkNotNull(fn, "fn must not be null"); - this.fn = fn; - this.signature = DoFnSignatures.getSignature(fn.getClass()); - checkArgument(signature.processElement().isSplittable(), "fn must be a splittable DoFn"); + public SplittableParDo(ParDo.BoundMulti parDo) { + checkNotNull(parDo, "parDo must not be null"); + this.parDo = parDo; + checkArgument( + DoFnSignatures.getSignature(parDo.getNewFn().getClass()).processElement().isSplittable(), + "fn must be a splittable DoFn"); } @Override - public PCollection apply(PCollection input) { - PCollection.IsBounded isFnBounded = signature.isBoundedPerElement(); + public PCollectionTuple apply(PCollection input) { + return applyTyped(input); + } + + private PCollectionTuple applyTyped(PCollection input) { + DoFn fn = parDo.getNewFn(); Coder restrictionCoder = - DoFnInvokers - .invokerFor(fn) + DoFnInvokers.invokerFor(fn) .invokeGetRestrictionCoder(input.getPipeline().getCoderRegistry()); + PCollection>> keyedWorkItems = + applySplitIntoKeyedWorkItems(input, fn, restrictionCoder); + return keyedWorkItems.apply( + "Process", + new ProcessElements<>( + fn, + input.getCoder(), + restrictionCoder, + input.getWindowingStrategy(), + parDo.getSideInputs(), + parDo.getMainOutputTag(), + parDo.getSideOutputTags())); + } + + private static + PCollection>> + applySplitIntoKeyedWorkItems( + PCollection input, + DoFn fn, + Coder restrictionCoder) { Coder> splitCoder = ElementAndRestrictionCoder.of(input.getCoder(), restrictionCoder); @@ -121,23 +157,133 @@ public PCollection apply(PCollection input) { WithKeys.of(new RandomUniqueKeyFn>())) .apply( "Group by key", - new GBKIntoKeyedWorkItems>()); + new GBKIntoKeyedWorkItems>()) + .setCoder( + KeyedWorkItemCoder.of( + StringUtf8Coder.of(), + splitCoder, + input.getWindowingStrategy().getWindowFn().windowCoder())); checkArgument( keyedWorkItems.getWindowingStrategy().getWindowFn() instanceof GlobalWindows, "GBKIntoKeyedWorkItems must produce a globally windowed collection, " + "but windowing strategy was: %s", keyedWorkItems.getWindowingStrategy()); - return keyedWorkItems - .apply( - "Process", - ParDo.of( - new ProcessFn( - fn, - input.getCoder(), - restrictionCoder, - input.getWindowingStrategy().getWindowFn().windowCoder()))) - .setIsBoundedInternal(input.isBounded().and(isFnBounded)) - .setWindowingStrategyInternal(input.getWindowingStrategy()); + return keyedWorkItems; + } + + /** + * Runner-specific primitive {@link GroupByKey GroupByKey-like} {@link PTransform} that produces + * {@link KeyedWorkItem KeyedWorkItems} so that downstream transforms can access state and timers. + * + *

    Unlike a real {@link GroupByKey}, ignores the input's windowing and triggering strategy and + * emits output immediately. + */ + public static class GBKIntoKeyedWorkItems + extends PTransform>, PCollection>> { + @Override + public PCollection> apply(PCollection> input) { + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded()); + } + } + + /** + * Runner-specific primitive {@link PTransform} that invokes the {@link DoFn.ProcessElement} + * method for a splittable {@link DoFn}. + */ + public static class ProcessElements + extends PTransform< + PCollection>>, + PCollectionTuple> { + private final DoFn fn; + private final Coder elementCoder; + private final Coder restrictionCoder; + private final WindowingStrategy windowingStrategy; + private final List> sideInputs; + private final TupleTag mainOutputTag; + private final TupleTagList sideOutputTags; + + /** + * @param fn the splittable {@link DoFn}. + * @param windowingStrategy the {@link WindowingStrategy} of the input collection. + * @param sideInputs list of side inputs that should be available to the {@link DoFn}. + * @param mainOutputTag {@link TupleTag Tag} of the {@link DoFn DoFn's} main output. + * @param sideOutputTags {@link TupleTagList Tags} of the {@link DoFn DoFn's} side outputs. + */ + public ProcessElements( + DoFn fn, + Coder elementCoder, + Coder restrictionCoder, + WindowingStrategy windowingStrategy, + List> sideInputs, + TupleTag mainOutputTag, + TupleTagList sideOutputTags) { + this.fn = fn; + this.elementCoder = elementCoder; + this.restrictionCoder = restrictionCoder; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + this.mainOutputTag = mainOutputTag; + this.sideOutputTags = sideOutputTags; + } + + public DoFn getFn() { + return fn; + } + + public List> getSideInputs() { + return sideInputs; + } + + public TupleTag getMainOutputTag() { + return mainOutputTag; + } + + public TupleTagList getSideOutputTags() { + return sideOutputTags; + } + + public ProcessFn newProcessFn(DoFn fn) { + return new SplittableParDo.ProcessFn<>( + fn, elementCoder, restrictionCoder, windowingStrategy.getWindowFn().windowCoder()); + } + + @Override + public PCollectionTuple apply( + PCollection>> + input) { + DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); + PCollectionTuple outputs = + PCollectionTuple.ofPrimitiveOutputsInternal( + input.getPipeline(), + TupleTagList.of(mainOutputTag).and(sideOutputTags.getAll()), + windowingStrategy, + input.isBounded().and(signature.isBoundedPerElement())); + + // Set output type descriptor similarly to how ParDo.BoundMulti does it. + outputs.get(mainOutputTag).setTypeDescriptorInternal(fn.getOutputTypeDescriptor()); + + return outputs; + } + + @Override + public Coder getDefaultOutputCoder( + PCollection>> + input, + TypedPValue output) + throws CannotProvideCoderException { + // Similar logic to ParDo.BoundMulti.getDefaultOutputCoder. + @SuppressWarnings("unchecked") + KeyedWorkItemCoder> kwiCoder = + (KeyedWorkItemCoder) input.getCoder(); + Coder inputCoder = + ((ElementAndRestrictionCoder) kwiCoder.getElementCoder()) + .getElementCoder(); + return input + .getPipeline() + .getCoderRegistry() + .getDefaultCoder(output.getTypeDescriptor(), fn.getInputTypeDescriptor(), inputCoder); + } } /** @@ -182,15 +328,11 @@ public void processElement(ProcessContext context) { * The heart of splittable {@link DoFn} execution: processes a single (element, restriction) pair * by creating a tracker for the restriction and checkpointing/resuming processing later if * necessary. - * - *

    TODO: This uses deprecated OldDoFn since DoFn does not provide access to state/timer - * internals. This should be rewritten to use the State - * and Timers API once it is available. */ @VisibleForTesting - static class ProcessFn< + public static class ProcessFn< InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> - extends OldDoFn>, OutputT> { + extends DoFn>, OutputT> { // Commit at least once every 10k output records. This keeps the watermark advancing // smoothly, and ensures that not too much work will have to be reprocessed in the event of // a crash. @@ -227,30 +369,56 @@ static class ProcessFn< */ private StateTag> restrictionTag; + private transient StateInternalsFactory stateInternalsFactory; + private transient TimerInternalsFactory timerInternalsFactory; + private transient OutputWindowedValue outputWindowedValue; + private final DoFn fn; private final Coder windowCoder; private transient DoFnInvoker invoker; - ProcessFn( + public ProcessFn( DoFn fn, Coder elementCoder, Coder restrictionCoder, Coder windowCoder) { this.fn = fn; + this.invoker = DoFnInvokers.invokerFor(fn); this.windowCoder = windowCoder; - elementTag = + this.elementTag = StateTags.value("element", WindowedValue.getFullCoder(elementCoder, this.windowCoder)); - restrictionTag = StateTags.value("restriction", restrictionCoder); + this.restrictionTag = StateTags.value("restriction", restrictionCoder); } - @Override - public void setup() throws Exception { - invoker = DoFnInvokers.invokerFor(fn); + public void setStateInternalsFactory(StateInternalsFactory stateInternalsFactory) { + this.stateInternalsFactory = stateInternalsFactory; } - @Override + public void setTimerInternalsFactory(TimerInternalsFactory timerInternalsFactory) { + this.timerInternalsFactory = timerInternalsFactory; + } + + public void setOutputWindowedValue(OutputWindowedValue outputWindowedValue) { + this.outputWindowedValue = outputWindowedValue; + } + + @StartBundle + public void startBundle(Context c) throws Exception { + invoker.invokeStartBundle(wrapContext(c)); + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + invoker.invokeFinishBundle(wrapContext(c)); + } + + @ProcessElement public void processElement(final ProcessContext c) { + StateInternals stateInternals = + stateInternalsFactory.stateInternalsForKey(c.element().key()); + TimerInternals timerInternals = timerInternalsFactory.timerInternalsForKey(c.element().key()); + // Initialize state (element and restriction) depending on whether this is the seed call. // The seed call is the first call for this element, which actually has the element. // Subsequent calls are timer firings and the element has to be retrieved from the state. @@ -258,17 +426,23 @@ public void processElement(final ProcessContext c) { boolean isSeedCall = (timer == null); StateNamespace stateNamespace = isSeedCall ? StateNamespaces.global() : timer.getNamespace(); ValueState> elementState = - c.windowingInternals().stateInternals().state(stateNamespace, elementTag); + stateInternals.state(stateNamespace, elementTag); ValueState restrictionState = - c.windowingInternals().stateInternals().state(stateNamespace, restrictionTag); + stateInternals.state(stateNamespace, restrictionTag); WatermarkHoldState holdState = - c.windowingInternals().stateInternals().state(stateNamespace, watermarkHoldTag); + stateInternals.state(stateNamespace, watermarkHoldTag); ElementAndRestriction, RestrictionT> elementAndRestriction; if (isSeedCall) { // The element and restriction are available in c.element(). + // elementsIterable() will, by construction of SplittableParDo, contain the same value + // potentially in several different windows. We implode this into a single WindowedValue + // in order to simplify the rest of the code and avoid iterating over elementsIterable() + // explicitly. The windows of this WindowedValue will be propagated to windows of the + // output. This is correct because a splittable DoFn is not allowed to inspect the window + // of its element. WindowedValue> windowedValue = - Iterables.getOnlyElement(c.element().elementsIterable()); + implodeWindows(c.element().elementsIterable()); WindowedValue element = windowedValue.withValue(windowedValue.getValue().element()); elementState.write(element); elementAndRestriction = @@ -290,7 +464,7 @@ public void processElement(final ProcessContext c) { DoFn.ProcessContinuation cont = invoker.invokeProcessElement( wrapTracker( - tracker, makeContext(c, elementAndRestriction.element(), tracker, residual))); + tracker, wrapContext(c, elementAndRestriction.element(), tracker, residual))); if (residual[0] == null) { // This means the call completed unsolicited, and the context produced by makeContext() // did not take a checkpoint. Take one now. @@ -307,19 +481,85 @@ public void processElement(final ProcessContext c) { } restrictionState.write(residual[0]); Instant futureOutputWatermark = cont.getWatermark(); - if (futureOutputWatermark != null) { - holdState.add(futureOutputWatermark); + if (futureOutputWatermark == null) { + futureOutputWatermark = elementAndRestriction.element().getTimestamp(); } + Instant wakeupTime = timerInternals.currentProcessingTime().plus(cont.resumeDelay()); + holdState.add(futureOutputWatermark); // Set a timer to continue processing this element. - TimerInternals timerInternals = c.windowingInternals().timerInternals(); timerInternals.setTimer( - TimerInternals.TimerData.of( - stateNamespace, - timerInternals.currentProcessingTime().plus(cont.resumeDelay()), - TimeDomain.PROCESSING_TIME)); + TimerInternals.TimerData.of(stateNamespace, wakeupTime, TimeDomain.PROCESSING_TIME)); + } + + /** + * Does the opposite of {@link WindowedValue#explodeWindows()} - creates a single {@link + * WindowedValue} from a collection of {@link WindowedValue}'s that is known to contain copies + * of the same value with the same timestamp, but different window sets. + * + *

    This is only legal to do because we know that {@link RandomUniqueKeyFn} created unique + * keys for every {@link ElementAndRestriction}, so if there's multiple {@link WindowedValue}'s + * for the same key, that means only that the windows of that {@link ElementAndRestriction} are + * being delivered separately rather than all at once. It is also legal to do because splittable + * {@link DoFn} is not allowed to access the window of its element, so we can propagate the full + * set of windows of its input to its output. + */ + private static + WindowedValue> implodeWindows( + Iterable>> values) { + WindowedValue> first = + Iterables.getFirst(values, null); + checkState(first != null, "Got a KeyedWorkItem with no elements and no timers"); + ImmutableList.Builder windows = ImmutableList.builder(); + for (WindowedValue> value : values) { + windows.addAll(value.getWindows()); + } + return WindowedValue.of( + first.getValue(), first.getTimestamp(), windows.build(), first.getPane()); + } + + private DoFn.Context wrapContext(final Context baseContext) { + return fn.new Context() { + @Override + public PipelineOptions getPipelineOptions() { + return baseContext.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + throwUnsupportedOutput(); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + throwUnsupportedOutput(); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throwUnsupportedOutput(); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + throwUnsupportedOutput(); + } + + @Override + protected Aggregator createAggregator( + String name, Combine.CombineFn combiner) { + return fn.createAggregator(name, combiner); + } + + private void throwUnsupportedOutput() { + throw new UnsupportedOperationException( + String.format( + "Splittable DoFn can only output from @%s", + ProcessElement.class.getSimpleName())); + } + }; } - private DoFn.ProcessContext makeContext( + private DoFn.ProcessContext wrapContext( final ProcessContext baseContext, final WindowedValue element, final TrackerT tracker, @@ -340,17 +580,14 @@ public PaneInfo pane() { } public void output(OutputT output) { - baseContext - .windowingInternals() - .outputWindowedValue( - output, element.getTimestamp(), element.getWindows(), element.getPane()); + outputWindowedValue.outputWindowedValue( + output, element.getTimestamp(), element.getWindows(), element.getPane()); noteOutput(); } public void outputWithTimestamp(OutputT output, Instant timestamp) { - baseContext - .windowingInternals() - .outputWindowedValue(output, timestamp, element.getWindows(), element.getPane()); + outputWindowedValue.outputWindowedValue( + output, timestamp, element.getWindows(), element.getPane()); noteOutput(); } @@ -370,17 +607,15 @@ public PipelineOptions getPipelineOptions() { } public void sideOutput(TupleTag tag, T output) { - // TODO: I'm not sure how to implement this correctly: there's no - // "internals.sideOutputWindowedValue". - throw new UnsupportedOperationException( - "Side outputs not yet supported by splittable DoFn"); + outputWindowedValue.sideOutputWindowedValue( + tag, output, element.getTimestamp(), element.getWindows(), element.getPane()); + noteOutput(); } public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - // TODO: I'm not sure how to implement this correctly: there's no - // "internals.sideOutputWindowedValue". - throw new UnsupportedOperationException( - "Side outputs not yet supported by splittable DoFn"); + outputWindowedValue.sideOutputWindowedValue( + tag, output, timestamp, element.getWindows(), element.getPane()); + noteOutput(); } @Override @@ -393,8 +628,7 @@ protected Aggregator createAggreg /** * Creates an {@link DoFnInvoker.ArgumentProvider} that provides the given tracker as well as - * the given - * {@link ProcessContext} (which is also provided when a {@link Context} is requested. + * the given {@link ProcessContext} (which is also provided when a {@link Context} is requested. */ private DoFnInvoker.ArgumentProvider wrapTracker( TrackerT tracker, DoFn.ProcessContext processContext) { diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index 29ff838b3fcc..990d8922e311 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -29,6 +29,7 @@ import java.io.Serializable; import java.util.Arrays; +import java.util.Collection; import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; @@ -38,6 +39,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -47,8 +49,13 @@ import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateInternalsFactory; +import org.apache.beam.sdk.util.state.TimerInternalsFactory; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -120,6 +127,12 @@ private static PCollection makeBoundedCollection(Pipeline pipeline) { .setIsBoundedInternal(PCollection.IsBounded.BOUNDED); } + private static final TupleTag MAIN_OUTPUT_TAG = new TupleTag() {}; + + private ParDo.BoundMulti makeParDo(DoFn fn) { + return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty()); + } + @Test public void testBoundednessForBoundedFn() { Pipeline pipeline = TestPipeline.create(); @@ -128,14 +141,15 @@ public void testBoundednessForBoundedFn() { "Applying a bounded SDF to a bounded collection produces a bounded collection", PCollection.IsBounded.BOUNDED, makeBoundedCollection(pipeline) - .apply("bounded to bounded", new SplittableParDo<>(boundedFn)) - .isBounded()); + .apply("bounded to bounded", new SplittableParDo<>(makeParDo(boundedFn))) + .get(MAIN_OUTPUT_TAG).isBounded()); assertEquals( "Applying a bounded SDF to an unbounded collection produces an unbounded collection", PCollection.IsBounded.UNBOUNDED, makeUnboundedCollection(pipeline) - .apply("bounded to unbounded", new SplittableParDo<>(boundedFn)) - .isBounded()); + .apply( + "bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn))) + .get(MAIN_OUTPUT_TAG).isBounded()); } @Test @@ -146,18 +160,27 @@ public void testBoundednessForUnboundedFn() { "Applying an unbounded SDF to a bounded collection produces a bounded collection", PCollection.IsBounded.UNBOUNDED, makeBoundedCollection(pipeline) - .apply("unbounded to bounded", new SplittableParDo<>(unboundedFn)) - .isBounded()); + .apply( + "unbounded to bounded", + new SplittableParDo<>(makeParDo(unboundedFn))) + .get(MAIN_OUTPUT_TAG).isBounded()); assertEquals( "Applying an unbounded SDF to an unbounded collection produces an unbounded collection", PCollection.IsBounded.UNBOUNDED, makeUnboundedCollection(pipeline) - .apply("unbounded to unbounded", new SplittableParDo<>(unboundedFn)) - .isBounded()); + .apply( + "unbounded to unbounded", + new SplittableParDo<>(makeParDo(unboundedFn))) + .get(MAIN_OUTPUT_TAG).isBounded()); } // ------------------------------- Tests for ProcessFn --------------------------------- + enum WindowExplosion { + EXPLODE_WINDOWS, + DO_NOT_EXPLODE_WINDOWS + } + /** * A helper for testing {@link SplittableParDo.ProcessFn} on 1 element (but possibly over multiple * {@link DoFn.ProcessElement} calls). @@ -179,6 +202,46 @@ private static class ProcessFnTester< new SplittableParDo.ProcessFn<>( fn, inputCoder, restrictionCoder, IntervalWindow.getCoder()); this.tester = DoFnTester.of(processFn); + processFn.setStateInternalsFactory( + new StateInternalsFactory() { + @Override + public StateInternals stateInternalsForKey(String key) { + return tester.getStateInternals(); + } + }); + processFn.setTimerInternalsFactory( + new TimerInternalsFactory() { + @Override + public TimerInternals timerInternalsForKey(String key) { + return tester.getTimerInternals(); + } + }); + processFn.setOutputWindowedValue( + new OutputWindowedValue() { + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + tester + .getMutableOutput(tester.getMainOutputTag()) + .add(WindowedValue.of(output, timestamp, windows, pane)); + } + + @Override + public void sideOutputWindowedValue( + TupleTag tag, + SideOutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + tester.getMutableOutput(tag).add(WindowedValue.of(output, timestamp, windows, pane)); + } + }); + // Do not clone since ProcessFn references non-serializable DoFnTester itself + // through the state/timer/output callbacks. + this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE); this.tester.startBundle(); this.tester.advanceProcessingTime(currentProcessingTime); @@ -192,12 +255,24 @@ void startElement(InputT element, RestrictionT restriction) throws Exception { ElementAndRestriction.of(element, restriction), currentProcessingTime, GlobalWindow.INSTANCE, - PaneInfo.ON_TIME_AND_ONLY_FIRING)); + PaneInfo.ON_TIME_AND_ONLY_FIRING), + WindowExplosion.DO_NOT_EXPLODE_WINDOWS); } - void startElement(WindowedValue> windowedValue) + void startElement( + WindowedValue> windowedValue, + WindowExplosion explosion) throws Exception { - tester.processElement(KeyedWorkItems.elementsWorkItem("key", Arrays.asList(windowedValue))); + switch (explosion) { + case EXPLODE_WINDOWS: + tester.processElement( + KeyedWorkItems.elementsWorkItem("key", windowedValue.explodeWindows())); + break; + case DO_NOT_EXPLODE_WINDOWS: + tester.processElement( + KeyedWorkItems.elementsWorkItem("key", Arrays.asList(windowedValue))); + break; + } } /** @@ -253,9 +328,6 @@ public void testTrivialProcessFnPropagatesOutputsWindowsAndTimestamp() throws Ex DoFn fn = new ToStringFn(); Instant base = Instant.now(); - ProcessFnTester tester = - new ProcessFnTester<>( - base, fn, BigEndianIntegerCoder.of(), SerializableCoder.of(SomeRestriction.class)); IntervalWindow w1 = new IntervalWindow( @@ -267,20 +339,26 @@ public void testTrivialProcessFnPropagatesOutputsWindowsAndTimestamp() throws Ex new IntervalWindow( base.minus(Duration.standardMinutes(3)), base.plus(Duration.standardMinutes(3))); - tester.startElement( - WindowedValue.of( - ElementAndRestriction.of(42, new SomeRestriction()), - base, - Arrays.asList(w1, w2, w3), - PaneInfo.ON_TIME_AND_ONLY_FIRING)); - - for (IntervalWindow w : new IntervalWindow[] {w1, w2, w3}) { - assertEquals( - Arrays.asList( - TimestampedValue.of("42a", base), - TimestampedValue.of("42b", base), - TimestampedValue.of("42c", base)), - tester.peekOutputElementsInWindow(w)); + for (WindowExplosion explosion : WindowExplosion.values()) { + ProcessFnTester tester = + new ProcessFnTester<>( + base, fn, BigEndianIntegerCoder.of(), SerializableCoder.of(SomeRestriction.class)); + tester.startElement( + WindowedValue.of( + ElementAndRestriction.of(42, new SomeRestriction()), + base, + Arrays.asList(w1, w2, w3), + PaneInfo.ON_TIME_AND_ONLY_FIRING), + explosion); + + for (IntervalWindow w : new IntervalWindow[] {w1, w2, w3}) { + assertEquals( + Arrays.asList( + TimestampedValue.of("42a", base), + TimestampedValue.of("42b", base), + TimestampedValue.of("42c", base)), + tester.peekOutputElementsInWindow(w)); + } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java index 680a971603dc..04becd7e1a28 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java @@ -17,48 +17,23 @@ */ package org.apache.beam.runners.direct; -import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.runners.core.GBKIntoKeyedWorkItems; -import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItemCoder; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -/** Provides an implementation of {@link GBKIntoKeyedWorkItems} for the Direct Runner. */ +/** + * Provides an implementation of {@link SplittableParDo.GBKIntoKeyedWorkItems} for the Direct + * Runner. + */ class DirectGBKIntoKeyedWorkItemsOverrideFactory implements PTransformOverrideFactory< PCollection>, PCollection>, - GBKIntoKeyedWorkItems> { + SplittableParDo.GBKIntoKeyedWorkItems> { @Override public PTransform>, PCollection>> - override(GBKIntoKeyedWorkItems transform) { - return new DirectGBKIntoKeyedWorkItems<>(transform.getName()); - } - - /** The Direct Runner specific implementation of {@link GBKIntoKeyedWorkItems}. */ - private static class DirectGBKIntoKeyedWorkItems - extends PTransform>, PCollection>> { - DirectGBKIntoKeyedWorkItems(String name) { - super(name); - } - - @Override - public PCollection> apply(PCollection> input) { - checkArgument(input.getCoder() instanceof KvCoder); - KvCoder kvCoder = (KvCoder) input.getCoder(); - return input - // TODO: Perhaps windowing strategy should instead be set by ReifyTAW, or by DGBKO - .setWindowingStrategyInternal(WindowingStrategy.globalDefault()) - .apply(new DirectGroupByKey.DirectGroupByKeyOnly()) - .setCoder( - KeyedWorkItemCoder.of( - kvCoder.getKeyCoder(), - kvCoder.getValueCoder(), - input.getWindowingStrategy().getWindowFn().windowCoder())); - } + override(SplittableParDo.GBKIntoKeyedWorkItems transform) { + return new DirectGroupByKey.DirectGroupByKeyOnly<>(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java index 219314a8572e..efee8016d784 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java @@ -65,7 +65,7 @@ public PCollection>> apply(PCollection> input) { KeyedWorkItemCoder.of( inputCoder.getKeyCoder(), inputCoder.getValueCoder(), - input.getWindowingStrategy().getWindowFn().windowCoder())) + inputWindowingStrategy.getWindowFn().windowCoder())) // Group each key's values by window, merging windows as needed. .apply( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index f71e109fc5de..82de9abe8bf0 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -30,7 +30,7 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; -import org.apache.beam.runners.core.GBKIntoKeyedWorkItems; +import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; @@ -88,7 +88,7 @@ public class DirectRunner extends PipelineRunner { .put(ParDo.Bound.class, new ParDoSingleViaMultiOverrideFactory()) .put(ParDo.BoundMulti.class, new ParDoMultiOverrideFactory()) .put( - GBKIntoKeyedWorkItems.class, + SplittableParDo.GBKIntoKeyedWorkItems.class, new DirectGBKIntoKeyedWorkItemsOverrideFactory()) .build(); @@ -307,8 +307,8 @@ public DirectPipelineResult run(Pipeline pipeline) { @SuppressWarnings("rawtypes") KeyedPValueTrackingVisitor keyedPValueVisitor = KeyedPValueTrackingVisitor.create( - ImmutableSet.>of( - GBKIntoKeyedWorkItems.class, + ImmutableSet.of( + SplittableParDo.GBKIntoKeyedWorkItems.class, DirectGroupByKeyOnly.class, DirectGroupAlsoByWindow.class)); pipeline.traverseTopologically(keyedPValueVisitor); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java index 67d957c54764..cd644a66585b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java @@ -56,9 +56,9 @@ private DoFnLifecycleManager(DoFn original) { thrownOnTeardown = new ConcurrentHashMap<>(); } - public DoFn get() throws Exception { + public DoFn get() throws Exception { Thread currentThread = Thread.currentThread(); - return outstanding.get(currentThread); + return (DoFn) outstanding.get(currentThread); } public void remove() throws Exception { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 750e5f1d07dc..504ddc462aab 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -20,7 +20,6 @@ import com.google.common.collect.ImmutableList; import java.io.Serializable; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -58,9 +57,9 @@ public static ParDoEvaluator create( Map, UncommittedBundle> outputBundles = new HashMap<>(); for (Map.Entry, PCollection> outputEntry : outputs.entrySet()) { outputBundles.put( - outputEntry.getKey(), - evaluationContext.createBundle(outputEntry.getValue())); + outputEntry.getKey(), evaluationContext.createBundle(outputEntry.getValue())); } + BundleOutputManager outputManager = BundleOutputManager.create(outputBundles); ReadyCheckingSideInputReader sideInputReader = evaluationContext.createSideInputReader(sideInputs); @@ -69,7 +68,7 @@ public static ParDoEvaluator create( evaluationContext.getPipelineOptions(), fn, sideInputReader, - BundleOutputManager.create(outputBundles), + outputManager, mainOutputTag, sideOutputTags, stepContext, @@ -85,12 +84,7 @@ public static ParDoEvaluator create( } return new ParDoEvaluator<>( - evaluationContext, - runner, - application, - aggregatorChanges, - outputBundles.values(), - stepContext); + evaluationContext, runner, application, aggregatorChanges, outputManager, stepContext); } //////////////////////////////////////////////////////////////////////////////////////////////// @@ -99,7 +93,7 @@ public static ParDoEvaluator create( private final PushbackSideInputDoFnRunner fnRunner; private final AppliedPTransform transform; private final AggregatorContainer.Mutator aggregatorChanges; - private final Collection> outputBundles; + private final BundleOutputManager outputManager; private final DirectStepContext stepContext; private final ImmutableList.Builder> unprocessedElements; @@ -109,17 +103,21 @@ private ParDoEvaluator( PushbackSideInputDoFnRunner fnRunner, AppliedPTransform transform, AggregatorContainer.Mutator aggregatorChanges, - Collection> outputBundles, + BundleOutputManager outputManager, DirectStepContext stepContext) { this.evaluationContext = evaluationContext; this.fnRunner = fnRunner; this.transform = transform; - this.outputBundles = outputBundles; + this.outputManager = outputManager; this.stepContext = stepContext; this.aggregatorChanges = aggregatorChanges; this.unprocessedElements = ImmutableList.builder(); } + public BundleOutputManager getOutputManager() { + return outputManager; + } + @Override public void processElement(WindowedValue element) { try { @@ -147,7 +145,7 @@ public TransformResult finishBundle() { resultBuilder = StepTransformResult.withoutHold(transform); } return resultBuilder - .addOutput(outputBundles) + .addOutput(outputManager.bundles.values()) .withTimerUpdate(stepContext.getTimerUpdate()) .withAggregatorChanges(aggregatorChanges) .addUnprocessedElements(unprocessedElements.build()) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 02e034aa74ee..ec5dc2c26400 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -57,6 +57,7 @@ public DoFnLifecycleManager load(DoFn key) throws Exception { public TransformEvaluator forApplication( AppliedPTransform application, CommittedBundle inputBundle) throws Exception { + @SuppressWarnings("unchecked") AppliedPTransform, PCollectionTuple, ParDo.BoundMulti> parDoApplication = (AppliedPTransform< @@ -93,13 +94,12 @@ public void cleanup() throws Exception { */ @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator createEvaluator( - AppliedPTransform, PCollectionTuple, ?> - application, - StructuralKey inputBundleKey, - DoFn doFn, - List> sideInputs, - TupleTag mainOutputTag, - List> sideOutputTags) + AppliedPTransform, PCollectionTuple, ?> application, + StructuralKey inputBundleKey, + DoFn doFn, + List> sideInputs, + TupleTag mainOutputTag, + List> sideOutputTags) throws Exception { String stepName = evaluationContext.getStepName(application); DirectStepContext stepContext = @@ -107,21 +107,40 @@ TransformEvaluator createEvaluator( .getExecutionContext(application, inputBundleKey) .getOrCreateStepContext(stepName, stepName); - DoFnLifecycleManager fnManager = fnClones.getUnchecked(doFn); + DoFnLifecycleManager fnManager = getManagerForCloneOf(doFn); + return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping( + createParDoEvaluator( + application, + sideInputs, + mainOutputTag, + sideOutputTags, + stepContext, + fnManager.get(), + fnManager), + fnManager); + } + + ParDoEvaluator createParDoEvaluator( + AppliedPTransform, PCollectionTuple, ?> application, + List> sideInputs, + TupleTag mainOutputTag, + List> sideOutputTags, + DirectStepContext stepContext, + DoFn fn, + DoFnLifecycleManager fnManager) + throws Exception { try { - return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping( - ParDoEvaluator.create( - evaluationContext, - stepContext, - application, - application.getInput().getWindowingStrategy(), - fnManager.get(), - sideInputs, - mainOutputTag, - sideOutputTags, - application.getOutput().getAll()), - fnManager); + return ParDoEvaluator.create( + evaluationContext, + stepContext, + application, + application.getInput().getWindowingStrategy(), + fn, + sideInputs, + mainOutputTag, + sideOutputTags, + application.getOutput().getAll()); } catch (Exception e) { try { fnManager.remove(); @@ -134,4 +153,8 @@ TransformEvaluator createEvaluator( throw e; } } + + public DoFnLifecycleManager getManagerForCloneOf(DoFn fn) { + return fnClones.getUnchecked(fn); + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 8db51592ba4b..9c9256dbff72 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -49,7 +49,7 @@ public PTransform, PCollectionTuple> override( DoFn fn = transform.getNewFn(); DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); if (signature.processElement().isSplittable()) { - return new SplittableParDo(fn); + return new SplittableParDo(transform); } else if (signature.stateDeclarations().size() > 0 || signature.timerDeclarations().size() > 0) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java new file mode 100644 index 000000000000..0eca71008abe --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -0,0 +1,144 @@ +/* + * 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.direct; + +import java.util.Collection; +import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.ElementAndRestriction; +import org.apache.beam.runners.core.OutputWindowedValue; +import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateInternalsFactory; +import org.apache.beam.sdk.util.state.TimerInternalsFactory; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Instant; + +class SplittableProcessElementsEvaluatorFactory + implements TransformEvaluatorFactory { + private final ParDoEvaluatorFactory< + KeyedWorkItem>, OutputT> + delegateFactory; + private final EvaluationContext evaluationContext; + + SplittableProcessElementsEvaluatorFactory(EvaluationContext evaluationContext) { + this.evaluationContext = evaluationContext; + this.delegateFactory = new ParDoEvaluatorFactory<>(evaluationContext); + } + + @Override + public TransformEvaluator forApplication( + AppliedPTransform application, CommittedBundle inputBundle) throws Exception { + @SuppressWarnings({"unchecked", "rawtypes"}) + TransformEvaluator evaluator = + (TransformEvaluator) + createEvaluator((AppliedPTransform) application, (CommittedBundle) inputBundle); + return evaluator; + } + + @Override + public void cleanup() throws Exception { + delegateFactory.cleanup(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private TransformEvaluator>> + createEvaluator( + AppliedPTransform< + PCollection>>, + PCollectionTuple, SplittableParDo.ProcessElements> + application, + CommittedBundle inputBundle) + throws Exception { + final SplittableParDo.ProcessElements transform = + application.getTransform(); + + DoFnLifecycleManager fnManager = delegateFactory.getManagerForCloneOf(transform.getFn()); + + SplittableParDo.ProcessFn processFn = + transform.newProcessFn(fnManager.get()); + + String stepName = evaluationContext.getStepName(application); + final DirectExecutionContext.DirectStepContext stepContext = + evaluationContext + .getExecutionContext(application, inputBundle.getKey()) + .getOrCreateStepContext(stepName, stepName); + + ParDoEvaluator>, OutputT> + parDoEvaluator = + delegateFactory.createParDoEvaluator( + application, + transform.getSideInputs(), + transform.getMainOutputTag(), + transform.getSideOutputTags().getAll(), + stepContext, + processFn, + fnManager); + + processFn.setStateInternalsFactory( + new StateInternalsFactory() { + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public StateInternals stateInternalsForKey(String key) { + return (StateInternals) stepContext.stateInternals(); + } + }); + + processFn.setTimerInternalsFactory( + new TimerInternalsFactory() { + @Override + public TimerInternals timerInternalsForKey(String key) { + return stepContext.timerInternals(); + } + }); + + final OutputManager outputManager = parDoEvaluator.getOutputManager(); + processFn.setOutputWindowedValue( + new OutputWindowedValue() { + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + outputManager.output( + transform.getMainOutputTag(), WindowedValue.of(output, timestamp, windows, pane)); + } + + @Override + public void sideOutputWindowedValue( + TupleTag tag, + SideOutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + } + }); + + return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnManager); + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index a4c462ab5ec8..1ddf9f4f501c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -61,6 +62,10 @@ public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt) .put( TestStreamEvaluatorFactory.DirectTestStreamFactory.DirectTestStream.class, new TestStreamEvaluatorFactory(ctxt)) + // Runner-specific primitive used in expansion of SplittableParDo + .put( + SplittableParDo.ProcessElements.class, + new SplittableProcessElementsEvaluatorFactory<>(ctxt)) .build(); return new TransformEvaluatorRegistry(primitives); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java index c164ce615548..f9e833f3fb5b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkState; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.Serializable; import java.util.ArrayList; @@ -32,20 +33,28 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.Keys; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; 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.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.MutableDateTime; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -66,6 +75,11 @@ static class OffsetRange implements Serializable { this.from = from; this.to = to; } + + @Override + public String toString() { + return "OffsetRange{" + "from=" + from + ", to=" + to + '}'; + } } private static class OffsetRangeTracker implements RestrictionTracker { @@ -140,11 +154,8 @@ public void process(ProcessContext c) { } } - @Ignore( - "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; " - + "It must be implemented as a primitive.") @Test - public void testPairWithIndexBasic() throws ClassNotFoundException { + public void testPairWithIndexBasic() { Pipeline p = TestPipeline.create(); p.getOptions().setRunner(DirectRunner.class); PCollection> res = @@ -167,11 +178,8 @@ public void testPairWithIndexBasic() throws ClassNotFoundException { p.run(); } - @Ignore( - "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; " - + "It must be implemented as a primitive.") @Test - public void testPairWithIndexWindowedTimestamped() throws ClassNotFoundException { + public void testPairWithIndexWindowedTimestamped() { // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps // of elements in the input collection. Pipeline p = TestPipeline.create(); @@ -228,4 +236,172 @@ public void testPairWithIndexWindowedTimestamped() throws ClassNotFoundException } p.run(); } + + private static class SDFWithSideInputsAndOutputs extends DoFn { + private final PCollectionView sideInput; + private final TupleTag sideOutput; + + private SDFWithSideInputsAndOutputs( + PCollectionView sideInput, TupleTag sideOutput) { + this.sideInput = sideInput; + this.sideOutput = sideOutput; + } + + @ProcessElement + public void process(ProcessContext c, OffsetRangeTracker tracker) { + checkState(tracker.tryClaim(tracker.currentRestriction().from)); + String side = c.sideInput(sideInput); + c.output("main:" + side + ":" + c.element()); + c.sideOutput(sideOutput, "side:" + side + ":" + c.element()); + } + + @GetInitialRestriction + public OffsetRange getInitialRestriction(Integer value) { + return new OffsetRange(0, 1); + } + + @NewTracker + public OffsetRangeTracker newTracker(OffsetRange range) { + return new OffsetRangeTracker(range); + } + } + + @Test + public void testSideInputsAndOutputs() throws Exception { + Pipeline p = TestPipeline.create(); + p.getOptions().setRunner(DirectRunner.class); + + PCollectionView sideInput = + p.apply("side input", Create.of("foo")).apply(View.asSingleton()); + TupleTag mainOutputTag = new TupleTag<>("main"); + TupleTag sideOutputTag = new TupleTag<>("side"); + + PCollectionTuple res = + p.apply("input", Create.of(0, 1, 2)) + .apply( + ParDo.of(new SDFWithSideInputsAndOutputs(sideInput, sideOutputTag)) + .withSideInputs(sideInput) + .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag))); + res.get(mainOutputTag).setCoder(StringUtf8Coder.of()); + res.get(sideOutputTag).setCoder(StringUtf8Coder.of()); + + PAssert.that(res.get(mainOutputTag)) + .containsInAnyOrder(Arrays.asList("main:foo:0", "main:foo:1", "main:foo:2")); + PAssert.that(res.get(sideOutputTag)) + .containsInAnyOrder(Arrays.asList("side:foo:0", "side:foo:1", "side:foo:2")); + + p.run(); + } + + @Test + public void testLateData() throws Exception { + Pipeline p = TestPipeline.create(); + p.getOptions().setRunner(DirectRunner.class); + + Instant base = Instant.now(); + + TestStream stream = + TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(base) + .addElements("aa") + .advanceWatermarkTo(base.plus(Duration.standardSeconds(5))) + .addElements(TimestampedValue.of("bb", base.minus(Duration.standardHours(1)))) + .advanceProcessingTime(Duration.standardHours(1)) + .advanceWatermarkToInfinity(); + + PCollection input = + p.apply(stream) + .apply( + Window.into(FixedWindows.of(Duration.standardMinutes(1))) + .withAllowedLateness(Duration.standardMinutes(1))); + + PCollection> afterSDF = + input + .apply(ParDo.of(new PairStringWithIndexToLength())) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection nonLate = + afterSDF.apply(GroupByKey.create()).apply(Keys.create()); + + // The splittable DoFn itself should not drop any data and act as pass-through. + PAssert.that(afterSDF) + .containsInAnyOrder( + Arrays.asList(KV.of("aa", 0), KV.of("aa", 1), KV.of("bb", 0), KV.of("bb", 1))); + + // But it should preserve the windowing strategy of the data, including allowed lateness: + // the follow-up GBK should drop the late data. + assertEquals(afterSDF.getWindowingStrategy(), input.getWindowingStrategy()); + PAssert.that(nonLate).containsInAnyOrder("aa"); + + p.run(); + } + + private static class SDFWithLifecycle extends DoFn { + private enum State { + BEFORE_SETUP, + OUTSIDE_BUNDLE, + INSIDE_BUNDLE, + TORN_DOWN + } + + private State state = State.BEFORE_SETUP; + + @ProcessElement + public void processElement(ProcessContext c, OffsetRangeTracker tracker) { + assertEquals(State.INSIDE_BUNDLE, state); + assertTrue(tracker.tryClaim(0)); + c.output(c.element()); + } + + @GetInitialRestriction + public OffsetRange getInitialRestriction(String value) { + return new OffsetRange(0, 1); + } + + @NewTracker + public OffsetRangeTracker newTracker(OffsetRange range) { + return new OffsetRangeTracker(range); + } + + @Setup + public void setUp() { + assertEquals(State.BEFORE_SETUP, state); + state = State.OUTSIDE_BUNDLE; + } + + @StartBundle + public void startBundle(Context c) { + assertEquals(State.OUTSIDE_BUNDLE, state); + state = State.INSIDE_BUNDLE; + } + + @FinishBundle + public void finishBundle(Context c) { + assertEquals(State.INSIDE_BUNDLE, state); + state = State.OUTSIDE_BUNDLE; + } + + @Teardown + public void tearDown() { + assertEquals(State.OUTSIDE_BUNDLE, state); + state = State.TORN_DOWN; + } + } + + @Test + public void testLifecycleMethods() throws Exception { + Pipeline p = TestPipeline.create(); + p.getOptions().setRunner(DirectRunner.class); + + PCollection res = + p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle())); + + PAssert.that(res).containsInAnyOrder("a", "b", "c"); + + p.run(); + } + + // TODO (https://issues.apache.org/jira/browse/BEAM-988): Test that Splittable DoFn + // emits output immediately (i.e. has a pass-through trigger) regardless of input's + // windowing/triggering strategy. } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 3f1a3f9ad5d0..7aabec980f1b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -120,6 +120,9 @@ public abstract class Context { * should be in, throwing an exception if the {@code WindowFn} attempts * to access any information about the input element. The output element * will have a timestamp of negative infinity. + * + *

    Note: A splittable {@link DoFn} is not allowed to output from + * {@link StartBundle} or {@link FinishBundle} methods. */ public abstract void output(OutputT output); @@ -142,6 +145,9 @@ public abstract class Context { * should be in, throwing an exception if the {@code WindowFn} attempts * to access any information about the input element except for the * timestamp. + * + *

    Note: A splittable {@link DoFn} is not allowed to output from + * {@link StartBundle} or {@link FinishBundle} methods. */ public abstract void outputWithTimestamp(OutputT output, Instant timestamp); @@ -168,6 +174,9 @@ public abstract class Context { * to access any information about the input element. The output element * will have a timestamp of negative infinity. * + *

    Note: A splittable {@link DoFn} is not allowed to output from + * {@link StartBundle} or {@link FinishBundle} methods. + * * @see ParDo#withOutputTags */ public abstract void sideOutput(TupleTag tag, T output); @@ -192,6 +201,9 @@ public abstract class Context { * to access any information about the input element except for the * timestamp. * + *

    Note: A splittable {@link DoFn} is not allowed to output from + * {@link StartBundle} or {@link FinishBundle} methods. + * * @see ParDo#withOutputTags */ public abstract void sideOutputWithTimestamp( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index daa8a0600e8a..0c6043fc310d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -140,6 +140,15 @@ public void setSideInput(PCollectionView sideInput, BoundedWindow window, windowValues.put(window, value); } + @SuppressWarnings("unchecked") + public StateInternals getStateInternals() { + return (StateInternals) stateInternals; + } + + public TimerInternals getTimerInternals() { + return timerInternals; + } + /** * When a {@link DoFnTester} should clone the {@link DoFn} under test and how it should manage * the lifecycle of the {@link DoFn}. @@ -321,7 +330,6 @@ public void finishBundle() throws Exception { * */ public List peekOutputElements() { - // TODO: Should we return an unmodifiable list? return Lists.transform( peekOutputElementsWithTimestamp(), new Function, OutputT>() { @@ -344,7 +352,7 @@ public OutputT apply(TimestampedValue input) { @Experimental public List> peekOutputElementsWithTimestamp() { // TODO: Should we return an unmodifiable list? - return Lists.transform(getOutput(mainOutputTag), + return Lists.transform(getImmutableOutput(mainOutputTag), new Function, TimestampedValue>() { @Override @SuppressWarnings("unchecked") @@ -370,7 +378,7 @@ public List> peekOutputElementsInWindow( TupleTag tag, BoundedWindow window) { ImmutableList.Builder> valuesBuilder = ImmutableList.builder(); - for (WindowedValue value : getOutput(tag)) { + for (WindowedValue value : getImmutableOutput(tag)) { if (value.getWindows().contains(window)) { valuesBuilder.add(TimestampedValue.of(value.getValue(), value.getTimestamp())); } @@ -384,7 +392,7 @@ public List> peekOutputElementsInWindow( * @see #peekOutputElements */ public void clearOutputElements() { - peekOutputElements().clear(); + getMutableOutput(mainOutputTag).clear(); } /** @@ -425,7 +433,7 @@ public List> takeOutputElementsWithTimestamp() { */ public List peekSideOutputElements(TupleTag tag) { // TODO: Should we return an unmodifiable list? - return Lists.transform(getOutput(tag), + return Lists.transform(getImmutableOutput(tag), new Function, T>() { @SuppressWarnings("unchecked") @Override @@ -441,7 +449,7 @@ public T apply(WindowedValue input) { * @see #peekSideOutputElements */ public void clearSideOutputElements(TupleTag tag) { - peekSideOutputElements(tag).clear(); + getMutableOutput(tag).clear(); } /** @@ -502,10 +510,25 @@ private AggregateT extractAggregatorValue( return combiner.extractOutput(accumulator); } - private List> getOutput(TupleTag tag) { + private List> getImmutableOutput(TupleTag tag) { @SuppressWarnings({"unchecked", "rawtypes"}) List> elems = (List) outputs.get(tag); - return MoreObjects.firstNonNull(elems, Collections.>emptyList()); + return ImmutableList.copyOf( + MoreObjects.firstNonNull(elems, Collections.>emptyList())); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public List> getMutableOutput(TupleTag tag) { + List> outputList = (List) outputs.get(tag); + if (outputList == null) { + outputList = new ArrayList<>(); + outputs.put(tag, (List) outputList); + } + return outputList; + } + + public TupleTag getMainOutputTag() { + return mainOutputTag; } private TestContext createContext(OldDoFn fn) { @@ -590,17 +613,7 @@ public void sideOutput(TupleTag tag, T output) { } public void noteOutput(TupleTag tag, WindowedValue output) { - getOutputList(tag).add(output); - } - - private List> getOutputList(TupleTag tag) { - @SuppressWarnings({"unchecked", "rawtypes"}) - List> outputList = (List) outputs.get(tag); - if (outputList == null) { - outputList = new ArrayList<>(); - outputs.put(tag, (List) outputList); - } - return outputList; + getMutableOutput(tag).add(output); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java new file mode 100644 index 000000000000..b9c3d5e200c4 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util.state; + +import java.io.Serializable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.util.TimerInternals; + +/** + * A factory for providing {@link TimerInternals} for a particular key. + * + *

    Because it will generally be embedded in a {@link org.apache.beam.sdk.transforms.DoFn DoFn}, + * albeit at execution time, it is marked {@link Serializable}. + */ +@Experimental(Kind.STATE) +public interface TimerInternalsFactory { + + /** Returns {@link TimerInternals} for the provided key. */ + TimerInternals timerInternalsForKey(K key); +} From 569e8d7085cf4e6effd379f23716202c6c5daf52 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 1 Dec 2016 13:19:14 -0800 Subject: [PATCH 054/279] Migrate TransformTreeNode to an Inner Class TransformTreeNode requires access to the hierarchy it is contained within, and generally cannot be separated from TransformHierarchy. It is primarily an implementation detail of TransformHierarchy, so can be relocated to within it. --- .../translation/ApexPipelineTranslator.java | 12 +- .../apex/translation/TranslationContext.java | 6 +- .../ConsumerTrackingPipelineVisitor.java | 12 +- .../runners/direct/DisplayDataValidator.java | 6 +- .../direct/KeyedPValueTrackingVisitor.java | 10 +- .../beam/runners/flink/FlinkRunner.java | 12 +- .../FlinkBatchPipelineTranslator.java | 14 +- .../FlinkStreamingPipelineTranslator.java | 16 +- .../PipelineTranslationOptimizer.java | 10 +- .../dataflow/DataflowPipelineTranslator.java | 8 +- .../beam/runners/dataflow/DataflowRunner.java | 10 +- .../runners/dataflow/DataflowRunnerTest.java | 4 +- .../dataflow/RecordingPipelineVisitor.java | 6 +- .../beam/runners/spark/SparkRunner.java | 21 +- .../beam/sdk/AggregatorPipelineExtractor.java | 6 +- .../java/org/apache/beam/sdk/Pipeline.java | 17 +- .../beam/sdk/runners/TransformHierarchy.java | 243 ++++++++++++++- .../beam/sdk/runners/TransformTreeNode.java | 282 ------------------ .../sdk/AggregatorPipelineExtractorTest.java | 20 +- .../sdk/runners/TransformHierarchyTest.java | 26 +- .../beam/sdk/runners/TransformTreeTest.java | 8 +- .../display/DisplayDataEvaluator.java | 8 +- 22 files changed, 343 insertions(+), 414 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index d38faf7a8775..8d6db84d021f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -19,17 +19,15 @@ package org.apache.beam.runners.apex.translation; import com.datatorrent.api.DAG; - import java.util.HashMap; import java.util.Map; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner.CreateApexPCollectionView; import org.apache.beam.runners.apex.translation.operators.ApexReadUnboundedInputOperator; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -84,18 +82,18 @@ public void translate(Pipeline pipeline, DAG dag) { } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { LOG.debug("entering composite transform {}", node.getTransform()); return CompositeBehavior.ENTER_TRANSFORM; } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { LOG.debug("leaving composite transform {}", node.getTransform()); } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { LOG.debug("visiting transform {}", node.getTransform()); PTransform transform = node.getTransform(); TransformTranslator translator = getTransformTranslator(transform.getClass()); @@ -108,7 +106,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { LOG.debug("visiting value {}", value); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index e016730ffef4..259afbdd7092 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -24,18 +24,16 @@ import com.datatorrent.api.Operator; import com.datatorrent.api.Operator.InputPort; import com.datatorrent.api.Operator.OutputPort; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.utils.ApexStateInternals; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; @@ -73,7 +71,7 @@ public InputT getViewInput(PCollectionView view) { this.pipelineOptions = pipelineOptions; } - public void setCurrentTransform(TransformTreeNode treeNode) { + public void setCurrentTransform(TransformHierarchy.Node treeNode) { this.currentTransform = AppliedPTransform.of(treeNode.getFullName(), treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform()); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java index 4fdfea071702..acfad16bf251 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollectionView; @@ -50,7 +50,7 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults { private boolean finalized = false; @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { checkState( !finalized, "Attempting to traverse a pipeline (node %s) with a %s " @@ -61,7 +61,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { checkState( !finalized, "Attempting to traverse a pipeline (node %s) with a %s which is already finalized", @@ -73,7 +73,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { toFinalize.removeAll(node.getInput().expand()); AppliedPTransform appliedTransform = getAppliedTransform(node); stepNames.put(appliedTransform, genStepName()); @@ -86,7 +86,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } } - private AppliedPTransform getAppliedTransform(TransformTreeNode node) { + private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { @SuppressWarnings({"rawtypes", "unchecked"}) AppliedPTransform application = AppliedPTransform.of( node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform()); @@ -94,7 +94,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { toFinalize.add(value); for (PValue expandedValue : value.expand()) { valueToConsumers.put(expandedValue, new ArrayList>()); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java index e09fe626f780..c77cb48f4f9b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; @@ -51,7 +51,7 @@ private static class Visitor extends Pipeline.PipelineVisitor.Defaults { private static final Visitor INSTANCE = new Visitor(); @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (!node.isRootNode()) { evaluateDisplayData(node.getTransform()); } @@ -60,7 +60,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { evaluateDisplayData(node.getTransform()); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 47b08576546a..5dc24c2b8581 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -22,7 +22,7 @@ import java.util.HashSet; import java.util.Set; import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; @@ -55,7 +55,7 @@ private KeyedPValueTrackingVisitor( } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { checkState( !finalized, "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)", @@ -65,7 +65,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { checkState( !finalized, "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)", @@ -79,10 +79,10 @@ public void leaveCompositeTransform(TransformTreeNode node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) {} + public void visitPrimitiveTransform(TransformHierarchy.Node node) {} @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { if (producesKeyedOutputs.contains(producer.getTransform().getClass())) { keyedValues.addAll(value.expand()); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index 488c170afec9..0b927347bfe9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -31,7 +31,6 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.Coder; @@ -41,7 +40,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -55,7 +54,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.client.program.DetachedEnvironment; import org.slf4j.Logger; @@ -259,18 +257,18 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); pipeline.traverseTopologically(new Pipeline.PipelineVisitor() { @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); } } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); } @@ -278,7 +276,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { } }); 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 f36be6b8c133..805c41cd7916 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 @@ -19,7 +19,7 @@ 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.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.flink.api.java.DataSet; @@ -63,7 +63,7 @@ public void translate(Pipeline pipeline) { // -------------------------------------------------------------------------------------------- @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { LOG.info(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); this.depth++; @@ -79,13 +79,13 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { this.depth--; LOG.info(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { LOG.info(genSpaces(this.depth) + "visitPrimitiveTransform- " + formatNodeName(node)); // get the transformation corresponding to the node we are @@ -103,7 +103,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { private > void applyBatchTransform( PTransform transform, - TransformTreeNode node, + TransformHierarchy.Node node, BatchTransformTranslator translator) { @SuppressWarnings("unchecked") @@ -128,7 +128,7 @@ public interface BatchTransformTranslator { /** * Returns a translator for the given node, if it is possible, otherwise null. */ - private static BatchTransformTranslator getTranslator(TransformTreeNode node) { + private static BatchTransformTranslator getTranslator(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); // Root of the graph is null @@ -139,7 +139,7 @@ private static BatchTransformTranslator getTranslator(TransformTreeNode node) return FlinkBatchTransformTranslators.getTranslator(transform); } - private static String formatNodeName(TransformTreeNode node) { + private static String formatNodeName(TransformHierarchy.Node node) { return node.toString().split("@")[1] + node.getTransform(); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java index e5c0d76fd47a..a07dc3d7bfe2 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; @@ -50,7 +50,7 @@ public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, Pipeline // -------------------------------------------------------------------------------------------- @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { LOG.info(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); this.depth++; @@ -69,13 +69,13 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { this.depth--; LOG.info(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { LOG.info(genSpaces(this.depth) + "visitPrimitiveTransform- " + formatNodeName(node)); // get the transformation corresponding to hte node we are // currently visiting and translate it into its Flink alternative. @@ -93,13 +93,13 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { // do nothing here } private > void applyStreamingTransform( PTransform transform, - TransformTreeNode node, + TransformHierarchy.Node node, StreamTransformTranslator translator) { @SuppressWarnings("unchecked") @@ -116,7 +116,7 @@ public void visitValue(PValue value, TransformTreeNode producer) { private > boolean applyCanTranslate( PTransform transform, - TransformTreeNode node, + TransformHierarchy.Node node, StreamTransformTranslator translator) { @SuppressWarnings("unchecked") @@ -151,7 +151,7 @@ boolean canTranslate(T transform, FlinkStreamingTranslationContext context) { } } - private static String formatNodeName(TransformTreeNode node) { + private static String formatNodeName(TransformHierarchy.Node node) { return node.toString().split("@")[1] + node.getTransform(); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java index 97d123cc0843..99f7cebae3a2 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java @@ -19,7 +19,7 @@ import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; import org.slf4j.Logger; @@ -52,15 +52,15 @@ public TranslationMode getTranslationMode() { } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { return CompositeBehavior.ENTER_TRANSFORM; } @Override - public void leaveCompositeTransform(TransformTreeNode node) {} + public void leaveCompositeTransform(TransformHierarchy.Node node) {} @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { Class transformClass = node.getTransform().getClass(); if (transformClass == Read.Unbounded.class) { LOG.info("Found {}. Switching to streaming execution.", transformClass); @@ -69,5 +69,5 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } @Override - public void visitValue(PValue value, TransformTreeNode producer) {} + public void visitValue(PValue value, TransformHierarchy.Node producer) {} } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 2af2caebd353..1cff42adf204 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -66,7 +66,7 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; @@ -517,11 +517,11 @@ public String getFullName(PTransform transform) { @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); TransformTranslator translator = getTransformTranslator(transform.getClass()); @@ -537,7 +537,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { LOG.debug("Checking translation of {}", value); if (value.getProducingTransformInternal() == null) { throw new RuntimeException( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 009985644155..6ed386aff0bc 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -119,7 +119,7 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -684,18 +684,18 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); pipeline.traverseTopologically(new PipelineVisitor() { @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); } } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); } @@ -703,7 +703,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { } }); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 3925ed48d3bc..5375c95001b5 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -81,7 +81,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.dataflow.TestCountingSource; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.TestPipeline; @@ -912,7 +912,7 @@ private static class CompositeTransformRecorder extends PipelineVisitor.Defaults private List> transforms = new ArrayList<>(); @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (node.getTransform() != null) { transforms.add(node.getTransform()); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java index 2090877f41ff..1d5a7f5a6794 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java @@ -20,7 +20,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; @@ -34,12 +34,12 @@ class RecordingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults { public final List values = new ArrayList<>(); @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { transforms.add(node.getTransform()); } @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { values.add(value); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 49e0113159aa..63f77c0b14a7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -32,7 +32,7 @@ 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.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; @@ -213,7 +213,7 @@ TranslationMode getTranslationMode() { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (translationMode.equals(TranslationMode.BATCH)) { Class transformClass = node.getTransform().getClass(); if (transformClass == Read.Unbounded.class) { @@ -239,7 +239,7 @@ public Evaluator(SparkPipelineTranslator translator, EvaluationContext ctxt) { } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (node.getTransform() != null) { @SuppressWarnings("unchecked") Class> transformClass = @@ -254,7 +254,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { return CompositeBehavior.ENTER_TRANSFORM; } - private boolean shouldDefer(TransformTreeNode node) { + private boolean shouldDefer(TransformHierarchy.Node node) { PInput input = node.getInput(); // if the input is not a PCollection, or it is but with non merging windows, don't defer. if (!(input instanceof PCollection) @@ -283,12 +283,12 @@ private boolean shouldDefer(TransformTreeNode node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { doVisitTransform(node); } > void - doVisitTransform(TransformTreeNode node) { + doVisitTransform(TransformHierarchy.Node node) { @SuppressWarnings("unchecked") TransformT transform = (TransformT) node.getTransform(); @SuppressWarnings("unchecked") @@ -304,11 +304,12 @@ public void visitPrimitiveTransform(TransformTreeNode node) { } /** - * Determine if this Node belongs to a Bounded branch of the pipeline, or Unbounded, and - * translate with the proper translator. + * Determine if this Node belongs to a Bounded branch of the pipeline, or Unbounded, and + * translate with the proper translator. */ - private > TransformEvaluator - translate(TransformTreeNode node, TransformT transform, Class transformClass) { + private > + TransformEvaluator translate( + TransformHierarchy.Node node, TransformT transform, Class transformClass) { //--- determine if node is bounded/unbounded. // usually, the input determines if the PCollection to apply the next transformation to // is BOUNDED or UNBOUNDED, meaning RDD/DStream. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java index 0e79abe9683e..d2130d06f46f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java @@ -23,7 +23,7 @@ import java.util.Collections; import java.util.Map; import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.AggregatorRetriever; import org.apache.beam.sdk.transforms.PTransform; @@ -62,7 +62,7 @@ public AggregatorVisitor(SetMultimap, PTransform> aggrega } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); addStepToAggregators(transform, getAggregators(transform)); } @@ -86,6 +86,6 @@ private void addStepToAggregators( } @Override - public void visitValue(PValue value, TransformTreeNode producer) {} + public void visitValue(PValue value, TransformHierarchy.Node producer) {} } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index c8a4439ba899..7a16f9d2dadf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; @@ -218,25 +217,25 @@ public interface PipelineVisitor { * *

    The return value controls whether or not child transforms are visited. */ - CompositeBehavior enterCompositeTransform(TransformTreeNode node); + CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node); /** * Called for each composite transform after all of its component transforms and their outputs * have been visited. */ - void leaveCompositeTransform(TransformTreeNode node); + void leaveCompositeTransform(TransformHierarchy.Node node); /** * Called for each primitive transform after all of its topological predecessors * and inputs have been visited. */ - void visitPrimitiveTransform(TransformTreeNode node); + void visitPrimitiveTransform(TransformHierarchy.Node node); /** * Called for each value after the transform that produced the value has been * visited. */ - void visitValue(PValue value, TransformTreeNode producer); + void visitValue(PValue value, TransformHierarchy.Node producer); /** * Control enum for indicating whether or not a traversal should process the contents of @@ -253,18 +252,18 @@ enum CompositeBehavior { */ class Defaults implements PipelineVisitor { @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { return CompositeBehavior.ENTER_TRANSFORM; } @Override - public void leaveCompositeTransform(TransformTreeNode node) { } + public void leaveCompositeTransform(TransformHierarchy.Node node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { } + public void visitPrimitiveTransform(TransformHierarchy.Node node) { } @Override - public void visitValue(PValue value, TransformTreeNode producer) { } + public void visitValue(PValue value, TransformHierarchy.Node producer) { } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index d3fd4974bbb9..662acc1f0dea 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -20,13 +20,18 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; +import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; @@ -37,13 +42,13 @@ * associated {@link PValue}s. */ public class TransformHierarchy { - private final TransformTreeNode root; - private final Map producers; + private final Node root; + private final Map producers; // Maintain a stack based on the enclosing nodes - private TransformTreeNode current; + private Node current; public TransformHierarchy() { - root = TransformTreeNode.root(this); + root = new Node(null, null, "", null); current = root; producers = new HashMap<>(); } @@ -58,20 +63,22 @@ public TransformHierarchy() { * * @return the added node */ - public TransformTreeNode pushNode(String name, PInput input, PTransform transform) { + public Node pushNode(String name, PInput input, PTransform transform) { checkNotNull( transform, "A %s must be provided for all Nodes", PTransform.class.getSimpleName()); checkNotNull( name, "A name must be provided for all %s Nodes", PTransform.class.getSimpleName()); checkNotNull( input, "An input must be provided for all %s Nodes", PTransform.class.getSimpleName()); - current = TransformTreeNode.subtransform(current, transform, name, input); + Node node = new Node(current, transform, name, input); + current.addComposite(node); + current = node; return current; } /** * Finish specifying all of the input {@link PValue PValues} of the current {@link - * TransformTreeNode}. Ensures that all of the inputs to the current node have been fully + * Node}. Ensures that all of the inputs to the current node have been fully * specified, and have been produced by a node in this graph. */ public void finishSpecifyingInput() { @@ -84,7 +91,7 @@ public void finishSpecifyingInput() { } /** - * Set the output of the current {@link TransformTreeNode}. If the output is new (setOutput has + * Set the output of the current {@link Node}. If the output is new (setOutput has * not previously been called with it as the parameter), the current node is set as the producer * of that {@link POutput}. * @@ -114,7 +121,7 @@ public void popNode() { checkState(current != null, "Can't pop the root node of a TransformHierarchy"); } - TransformTreeNode getProducer(PValue produced) { + Node getProducer(PValue produced) { return producers.get(produced); } @@ -122,10 +129,10 @@ TransformTreeNode getProducer(PValue produced) { * Returns all producing transforms for the {@link PValue PValues} contained * in {@code output}. */ - List getProducingTransforms(POutput output) { - List producingTransforms = new ArrayList<>(); + List getProducingTransforms(POutput output) { + List producingTransforms = new ArrayList<>(); for (PValue value : output.expand()) { - TransformTreeNode producer = getProducer(value); + Node producer = getProducer(value); if (producer != null) { producingTransforms.add(producer); } @@ -139,7 +146,217 @@ public Set visit(PipelineVisitor visitor) { return visitedValues; } - public TransformTreeNode getCurrent() { + public Node getCurrent() { return current; } + + /** + * Provides internal tracking of transform relationships with helper methods + * for initialization and ordered visitation. + */ + public class Node { + private final Node enclosingNode; + // The PTransform for this node, which may be a composite PTransform. + // The root of a TransformHierarchy is represented as a Node + // with a null transform field. + private final PTransform transform; + + private final String fullName; + + // Nodes for sub-transforms of a composite transform. + private final Collection parts = new ArrayList<>(); + + // Input to the transform, in unexpanded form. + private final PInput input; + + // TODO: track which outputs need to be exported to parent. + // Output of the transform, in unexpanded form. + private POutput output; + + @VisibleForTesting + boolean finishedSpecifying = false; + + /** + * Creates a new Node with the given parent and transform. + * + *

    EnclosingNode and transform may both be null for a root-level node, which holds all other + * nodes. + * + * @param enclosingNode the composite node containing this node + * @param transform the PTransform tracked by this node + * @param fullName the fully qualified name of the transform + * @param input the unexpanded input to the transform + */ + private Node( + @Nullable Node enclosingNode, + @Nullable PTransform transform, + String fullName, + @Nullable PInput input) { + this.enclosingNode = enclosingNode; + this.transform = transform; + this.fullName = fullName; + this.input = input; + } + + /** + * Returns the transform associated with this transform node. + */ + public PTransform getTransform() { + return transform; + } + + /** + * Returns the enclosing composite transform node, or null if there is none. + */ + public Node getEnclosingNode() { + return enclosingNode; + } + + /** + * Adds a composite operation to the transform node. + * + *

    As soon as a node is added, the transform node is considered a + * composite operation instead of a primitive transform. + */ + public void addComposite(Node node) { + parts.add(node); + } + + /** + * Returns true if this node represents a composite transform that does not perform processing + * of its own, but merely encapsulates a sub-pipeline (which may be empty). + * + *

    Note that a node may be composite with no sub-transforms if it returns its input directly + * extracts a component of a tuple, or other operations that occur at pipeline assembly time. + */ + public boolean isCompositeNode() { + return !parts.isEmpty() || isRootNode() || returnsOthersOutput(); + } + + private boolean returnsOthersOutput() { + PTransform transform = getTransform(); + if (output != null) { + for (PValue outputValue : output.expand()) { + if (!getProducer(outputValue).getTransform().equals(transform)) { + return true; + } + } + } + return false; + } + + public boolean isRootNode() { + return transform == null; + } + + public String getFullName() { + return fullName; + } + + /** + * Returns the transform input, in unexpanded form. + */ + public PInput getInput() { + return input; + } + + /** + * Adds an output to the transform node. + */ + public void setOutput(POutput output) { + checkState(!finishedSpecifying); + checkState( + this.output == null, "Tried to specify more than one output for %s", getFullName()); + checkNotNull(output, "Tried to set the output of %s to null", getFullName()); + this.output = output; + + // Validate that a primitive transform produces only primitive output, and a composite + // transform does not produce primitive output. + Set outputProducers = new HashSet<>(); + for (PValue outputValue : output.expand()) { + outputProducers.add(getProducer(outputValue)); + } + if (outputProducers.contains(this) && outputProducers.size() != 1) { + Set otherProducerNames = new HashSet<>(); + for (Node outputProducer : outputProducers) { + if (outputProducer != this) { + otherProducerNames.add(outputProducer.getFullName()); + } + } + throw new IllegalArgumentException( + String.format( + "Output of transform [%s] contains a %s produced by it as well as other " + + "Transforms. A primitive transform must produce all of its outputs, and " + + "outputs of a composite transform must be produced by a component transform " + + "or be part of the input." + + "%n Other Outputs: %s" + + "%n Other Producers: %s", + getFullName(), POutput.class.getSimpleName(), output.expand(), otherProducerNames)); + } + } + + /** Returns the transform output, in unexpanded form. */ + public POutput getOutput() { + return output; + } + + AppliedPTransform toAppliedPTransform() { + return AppliedPTransform.of( + getFullName(), getInput(), getOutput(), (PTransform) getTransform()); + } + /** + * Visit the transform node. + * + *

    Provides an ordered visit of the input values, the primitive transform (or child nodes for + * composite transforms), then the output values. + */ + public void visit(PipelineVisitor visitor, Set visitedValues) { + if (!finishedSpecifying) { + finishSpecifying(); + } + + if (!isRootNode()) { + // Visit inputs. + for (PValue inputValue : input.expand()) { + if (visitedValues.add(inputValue)) { + visitor.visitValue(inputValue, getProducer(inputValue)); + } + } + } + + if (isCompositeNode()) { + PipelineVisitor.CompositeBehavior recurse = visitor.enterCompositeTransform(this); + + if (recurse.equals(CompositeBehavior.ENTER_TRANSFORM)) { + for (Node child : parts) { + child.visit(visitor, visitedValues); + } + } + visitor.leaveCompositeTransform(this); + } else { + visitor.visitPrimitiveTransform(this); + } + + if (!isRootNode()) { + // Visit outputs. + for (PValue pValue : output.expand()) { + if (visitedValues.add(pValue)) { + visitor.visitValue(pValue, this); + } + } + } + } + + /** + * Finish specifying a transform. + * + *

    All inputs are finished first, then the transform, then all outputs. + */ + public void finishSpecifying() { + if (finishedSpecifying) { + return; + } + finishedSpecifying = true; + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java deleted file mode 100644 index ea94bd958d5d..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java +++ /dev/null @@ -1,282 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.runners; - -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - -import com.google.common.annotations.VisibleForTesting; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.PValue; - -/** - * Provides internal tracking of transform relationships with helper methods - * for initialization and ordered visitation. - */ -public class TransformTreeNode { - private final TransformHierarchy hierarchy; - private final TransformTreeNode enclosingNode; - - // The PTransform for this node, which may be a composite PTransform. - // The root of a TransformHierarchy is represented as a TransformTreeNode - // with a null transform field. - private final PTransform transform; - - private final String fullName; - - // Nodes for sub-transforms of a composite transform. - private final Collection parts = new ArrayList<>(); - - // Input to the transform, in unexpanded form. - private final PInput input; - - // TODO: track which outputs need to be exported to parent. - // Output of the transform, in unexpanded form. - private POutput output; - - @VisibleForTesting - boolean finishedSpecifying = false; - - /** - * Create a root {@link TransformTreeNode}. This transform is the root of the provided {@link - * TransformHierarchy} - it has no enclosing node, no {@link PTransform}, no {@link PInput input}, - * no {@link POutput output}, and an empty name. It contains all {@link PTransform transforms} - * within a {@link Pipeline} as component transforms. - */ - public static TransformTreeNode root(TransformHierarchy hierarchy) { - return new TransformTreeNode(hierarchy, null, null, "", null); - } - - /** - * Create a subtransform of the provided {@link TransformTreeNode node}. The enclosing node is a - * composite that contains this transform. - * - *

    The returned node is a component node of the enclosing node. - */ - public static TransformTreeNode subtransform( - TransformTreeNode enclosing, PTransform transform, String fullName, PInput input) { - checkNotNull(enclosing); - checkNotNull(transform); - checkNotNull(fullName); - checkNotNull(input); - TransformTreeNode node = - new TransformTreeNode(enclosing.hierarchy, enclosing, transform, fullName, input); - enclosing.addComposite(node); - return node; - } - - /** - * Creates a new TransformTreeNode with the given parent and transform. - * - *

    EnclosingNode and transform may both be null for a root-level node, which holds all other - * nodes. - * - * @param enclosingNode the composite node containing this node - * @param transform the PTransform tracked by this node - * @param fullName the fully qualified name of the transform - * @param input the unexpanded input to the transform - */ - private TransformTreeNode( - TransformHierarchy hierarchy, - @Nullable TransformTreeNode enclosingNode, - @Nullable PTransform transform, - String fullName, - @Nullable PInput input) { - this.hierarchy = hierarchy; - this.enclosingNode = enclosingNode; - this.transform = transform; - this.fullName = fullName; - this.input = input; - } - - /** - * Returns the transform associated with this transform node. - */ - public PTransform getTransform() { - return transform; - } - - /** - * Returns the enclosing composite transform node, or null if there is none. - */ - public TransformTreeNode getEnclosingNode() { - return enclosingNode; - } - - /** - * Adds a composite operation to the transform node. - * - *

    As soon as a node is added, the transform node is considered a - * composite operation instead of a primitive transform. - */ - public void addComposite(TransformTreeNode node) { - parts.add(node); - } - - /** - * Returns true if this node represents a composite transform that does not perform processing of - * its own, but merely encapsulates a sub-pipeline (which may be empty). - * - *

    Note that a node may be composite with no sub-transforms if it returns its input directly - * extracts a component of a tuple, or other operations that occur at pipeline assembly time. - */ - public boolean isCompositeNode() { - return !parts.isEmpty() || isRootNode() || returnsOthersOutput(); - } - - private boolean returnsOthersOutput() { - PTransform transform = getTransform(); - if (output != null) { - for (PValue outputValue : output.expand()) { - if (!hierarchy.getProducer(outputValue).getTransform().equals(transform)) { - return true; - } - } - } - return false; - } - - public boolean isRootNode() { - return transform == null; - } - - public String getFullName() { - return fullName; - } - - /** - * Returns the transform input, in unexpanded form. - */ - public PInput getInput() { - return input; - } - - /** - * Adds an output to the transform node. - */ - public void setOutput(POutput output) { - checkState(!finishedSpecifying); - checkState(this.output == null, "Tried to specify more than one output for %s", getFullName()); - checkNotNull(output, "Tried to set the output of %s to null", getFullName()); - this.output = output; - - // Validate that a primitive transform produces only primitive output, and a composite transform - // does not produce primitive output. - Set outputProducers = new HashSet<>(); - for (PValue outputValue : output.expand()) { - outputProducers.add(hierarchy.getProducer(outputValue)); - } - if (outputProducers.contains(this) && outputProducers.size() != 1) { - Set otherProducerNames = new HashSet<>(); - for (TransformTreeNode outputProducer : outputProducers) { - if (outputProducer != this) { - otherProducerNames.add(outputProducer.getFullName()); - } - } - throw new IllegalArgumentException( - String.format( - "Output of transform [%s] contains a %s produced by it as well as other Transforms. " - + "A primitive transform must produce all of its outputs, and outputs of a " - + "composite transform must be produced by a component transform or be part of" - + "the input." - + "%n Other Outputs: %s" - + "%n Other Producers: %s", - getFullName(), POutput.class.getSimpleName(), output.expand(), otherProducerNames)); - } - } - - /** - * Returns the transform output, in unexpanded form. - */ - public POutput getOutput() { - return output; - } - - AppliedPTransform toAppliedPTransform() { - return AppliedPTransform.of( - getFullName(), getInput(), getOutput(), (PTransform) getTransform()); - } - /** - * Visit the transform node. - * - *

    Provides an ordered visit of the input values, the primitive - * transform (or child nodes for composite transforms), then the - * output values. - */ - public void visit(PipelineVisitor visitor, - Set visitedValues) { - if (!finishedSpecifying) { - finishSpecifying(); - } - - if (!isRootNode()) { - // Visit inputs. - for (PValue inputValue : input.expand()) { - if (visitedValues.add(inputValue)) { - visitor.visitValue(inputValue, hierarchy.getProducer(inputValue)); - } - } - } - - if (isCompositeNode()) { - PipelineVisitor.CompositeBehavior recurse = visitor.enterCompositeTransform(this); - - if (recurse.equals(CompositeBehavior.ENTER_TRANSFORM)) { - for (TransformTreeNode child : parts) { - child.visit(visitor, visitedValues); - } - } - visitor.leaveCompositeTransform(this); - } else { - visitor.visitPrimitiveTransform(this); - } - - if (!isRootNode()) { - // Visit outputs. - for (PValue pValue : output.expand()) { - if (visitedValues.add(pValue)) { - visitor.visitValue(pValue, this); - } - } - } - } - - /** - * Finish specifying a transform. - * - *

    All inputs are finished first, then the transform, then - * all outputs. - */ - public void finishSpecifying() { - if (finishedSpecifying) { - return; - } - finishedSpecifying = true; - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java index 335d81f47bbd..b4de768b5998 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java @@ -30,7 +30,7 @@ import java.util.Map; import java.util.UUID; import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Max; @@ -73,7 +73,7 @@ public void testGetAggregatorStepsWithParDoBoundExtractsSteps() { Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinIntegerFn()); - TransformTreeNode transformNode = mock(TransformTreeNode.class); + TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class); when(transformNode.getTransform()).thenReturn(bound); doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode))) @@ -101,7 +101,7 @@ public void testGetAggregatorStepsWithParDoBoundMultiExtractsSteps() { Aggregator aggregatorOne = fn.addAggregator(new Max.MaxLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn()); - TransformTreeNode transformNode = mock(TransformTreeNode.class); + TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class); when(transformNode.getTransform()).thenReturn(bound); doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode))) @@ -132,9 +132,9 @@ public void testGetAggregatorStepsWithOneAggregatorInMultipleStepsAddsSteps() { Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn()); - TransformTreeNode transformNode = mock(TransformTreeNode.class); + TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class); when(transformNode.getTransform()).thenReturn(bound); - TransformTreeNode otherTransformNode = mock(TransformTreeNode.class); + TransformHierarchy.Node otherTransformNode = mock(TransformHierarchy.Node.class); when(otherTransformNode.getTransform()).thenReturn(otherBound); doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode, otherTransformNode))) @@ -172,9 +172,9 @@ public void testGetAggregatorStepsWithDifferentStepsAddsSteps() { when(otherBound.getFn()).thenReturn(otherFn); - TransformTreeNode transformNode = mock(TransformTreeNode.class); + TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class); when(transformNode.getTransform()).thenReturn(bound); - TransformTreeNode otherTransformNode = mock(TransformTreeNode.class); + TransformHierarchy.Node otherTransformNode = mock(TransformHierarchy.Node.class); when(otherTransformNode.getTransform()).thenReturn(otherBound); doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode, otherTransformNode))) @@ -192,16 +192,16 @@ public void testGetAggregatorStepsWithDifferentStepsAddsSteps() { } private static class VisitNodesAnswer implements Answer { - private final List nodes; + private final List nodes; - public VisitNodesAnswer(List nodes) { + public VisitNodesAnswer(List nodes) { this.nodes = nodes; } @Override public Object answer(InvocationOnMock invocation) throws Throwable { PipelineVisitor visitor = (PipelineVisitor) invocation.getArguments()[0]; - for (TransformTreeNode node : nodes) { + for (TransformHierarchy.Node node : nodes) { visitor.visitPrimitiveTransform(node); } return null; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index 3bf6d6440122..f4488f4f8cd3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -77,8 +77,8 @@ public void pushWithoutPushFails() { @Test public void pushThenPopSucceeds() { - TransformTreeNode root = hierarchy.getCurrent(); - TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); + TransformHierarchy.Node root = hierarchy.getCurrent(); + TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); assertThat(hierarchy.getCurrent(), equalTo(node)); hierarchy.popNode(); assertThat(node.finishedSpecifying, is(true)); @@ -90,12 +90,12 @@ public void emptyCompositeSucceeds() { PCollection created = PCollection.createPrimitiveOutputInternal( pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); - TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); + TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); hierarchy.setOutput(created); hierarchy.popNode(); PCollectionList pcList = PCollectionList.of(created); - TransformTreeNode emptyTransform = + TransformHierarchy.Node emptyTransform = hierarchy.pushNode( "Extract", pcList, @@ -149,7 +149,7 @@ public PCollectionList apply(PCollectionList input) { @Test public void visitVisitsAllPushed() { - TransformTreeNode root = hierarchy.getCurrent(); + TransformHierarchy.Node root = hierarchy.getCurrent(); PBegin begin = PBegin.in(pipeline); Create.Values create = Create.of(1L); @@ -170,7 +170,7 @@ public Long apply(Long input) { PCollection.createPrimitiveOutputInternal( pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); - TransformTreeNode compositeNode = hierarchy.pushNode("Create", begin, create); + TransformHierarchy.Node compositeNode = hierarchy.pushNode("Create", begin, create); assertThat(hierarchy.getCurrent(), equalTo(compositeNode)); assertThat(compositeNode.getInput(), Matchers.equalTo(begin)); assertThat(compositeNode.getTransform(), Matchers.>equalTo(create)); @@ -178,7 +178,7 @@ public Long apply(Long input) { assertThat(compositeNode.getOutput(), nullValue()); assertThat(compositeNode.getEnclosingNode().isRootNode(), is(true)); - TransformTreeNode primitiveNode = hierarchy.pushNode("Create/Read", begin, read); + TransformHierarchy.Node primitiveNode = hierarchy.pushNode("Create/Read", begin, read); assertThat(hierarchy.getCurrent(), equalTo(primitiveNode)); hierarchy.setOutput(created); hierarchy.popNode(); @@ -194,30 +194,30 @@ public Long apply(Long input) { assertThat(hierarchy.getProducer(created), equalTo(primitiveNode)); hierarchy.popNode(); - TransformTreeNode otherPrimitive = hierarchy.pushNode("ParDo", created, map); + TransformHierarchy.Node otherPrimitive = hierarchy.pushNode("ParDo", created, map); hierarchy.setOutput(mapped); hierarchy.popNode(); - final Set visitedCompositeNodes = new HashSet<>(); - final Set visitedPrimitiveNodes = new HashSet<>(); + final Set visitedCompositeNodes = new HashSet<>(); + final Set visitedPrimitiveNodes = new HashSet<>(); final Set visitedValuesInVisitor = new HashSet<>(); Set visitedValues = hierarchy.visit( new PipelineVisitor.Defaults() { @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { visitedCompositeNodes.add(node); return CompositeBehavior.ENTER_TRANSFORM; } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { visitedPrimitiveNodes.add(node); } @Override - public void visitValue(PValue value, TransformTreeNode producer) { + public void visitValue(PValue value, TransformHierarchy.Node producer) { visitedValuesInVisitor.add(value); } }); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java index b95fa70fe785..a81fb1ae7b2e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java @@ -51,7 +51,7 @@ import org.junit.runners.JUnit4; /** - * Tests for {@link TransformTreeNode} and {@link TransformHierarchy}. + * Tests for {@link TransformHierarchy.Node} and {@link TransformHierarchy}. */ @RunWith(JUnit4.class) public class TransformTreeTest { @@ -128,7 +128,7 @@ public void testCompositeCapture() throws Exception { p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() { @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); if (transform instanceof Sample.SampleAny) { assertTrue(visited.add(TransformsSeen.SAMPLE_ANY)); @@ -144,7 +144,7 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); if (transform instanceof Sample.SampleAny) { assertTrue(left.add(TransformsSeen.SAMPLE_ANY)); @@ -152,7 +152,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); // Pick is a composite, should not be visited here. assertThat(transform, not(instanceOf(Sample.SampleAny.class))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java index b758ed6087fc..31ac91303a96 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java @@ -24,7 +24,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.runners.TransformTreeNode; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; @@ -143,7 +143,7 @@ Set getPrimitivesDisplayData() { } @Override - public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { if (Objects.equals(root, node.getTransform())) { inCompositeRoot = true; } @@ -152,14 +152,14 @@ public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { } @Override - public void leaveCompositeTransform(TransformTreeNode node) { + public void leaveCompositeTransform(TransformHierarchy.Node node) { if (Objects.equals(root, node.getTransform())) { inCompositeRoot = false; } } @Override - public void visitPrimitiveTransform(TransformTreeNode node) { + public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (inCompositeRoot || Objects.equals(root, node.getTransform())) { displayData.add(DisplayData.from(node.getTransform())); } From 7c5c7910c0b8cba8623a1c49fc24c51ea691dac3 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 1 Dec 2016 13:22:11 -0800 Subject: [PATCH 055/279] Reduce the visibility of TransformHierarchy Node Mutators These mutators should not be accessible when visiting the nodes. --- .../org/apache/beam/sdk/runners/TransformHierarchy.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 662acc1f0dea..e9829cc56099 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -263,7 +263,7 @@ public PInput getInput() { /** * Adds an output to the transform node. */ - public void setOutput(POutput output) { + private void setOutput(POutput output) { checkState(!finishedSpecifying); checkState( this.output == null, "Tried to specify more than one output for %s", getFullName()); @@ -304,13 +304,14 @@ public POutput getOutput() { return AppliedPTransform.of( getFullName(), getInput(), getOutput(), (PTransform) getTransform()); } + /** * Visit the transform node. * *

    Provides an ordered visit of the input values, the primitive transform (or child nodes for * composite transforms), then the output values. */ - public void visit(PipelineVisitor visitor, Set visitedValues) { + private void visit(PipelineVisitor visitor, Set visitedValues) { if (!finishedSpecifying) { finishSpecifying(); } @@ -352,7 +353,7 @@ public void visit(PipelineVisitor visitor, Set visitedValues) { * *

    All inputs are finished first, then the transform, then all outputs. */ - public void finishSpecifying() { + private void finishSpecifying() { if (finishedSpecifying) { return; } From 887b357f7dff9fe10914b32dc69d32f0716fa237 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 30 Nov 2016 12:55:45 -0800 Subject: [PATCH 056/279] Category for tests using splittable DoFn --- runners/apex/pom.xml | 1 + runners/flink/runner/pom.xml | 1 + runners/google-cloud-dataflow-java/pom.xml | 1 + runners/spark/pom.xml | 1 + .../beam/sdk/testing/UsesSplittableParDo.java | 25 ++++++++++++++++++ .../sdk/transforms}/SplittableDoFnTest.java | 26 +++++++------------ 6 files changed, 39 insertions(+), 16 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java rename {runners/direct-java/src/test/java/org/apache/beam/runners/direct => sdks/java/core/src/test/java/org/apache/beam/sdk/transforms}/SplittableDoFnTest.java (95%) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 84185b8ec592..983781d3803f 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -186,6 +186,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo + org.apache.beam.sdk.testing.UsesSplittableParDo none true diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 18bf6a72a7de..3e3dd7ec0ac5 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -54,6 +54,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo + org.apache.beam.sdk.testing.UsesSplittableParDo none true diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 59276e486022..85474995c1d2 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -78,6 +78,7 @@ runnable-on-service-tests org.apache.beam.sdk.testing.UsesStatefulParDo + org.apache.beam.sdk.testing.UsesSplittableParDo org.apache.beam.sdk.transforms.FlattenTest diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index da7a72a37a5b..dc000bfa8574 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -73,6 +73,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo + org.apache.beam.sdk.testing.UsesSplittableParDo 1 false true diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java new file mode 100644 index 000000000000..209936f3614f --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.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; + +import org.apache.beam.sdk.transforms.ParDo; + +/** + * Category tag for validation tests which utilize splittable {@link ParDo}. + */ +public interface UsesSplittableParDo {} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java similarity index 95% rename from runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java index f9e833f3fb5b..82bd3a3c14c9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.direct; +package org.apache.beam.sdk.transforms; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -32,14 +32,10 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; 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.TestStream; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.Keys; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.testing.UsesSplittableParDo; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -56,14 +52,12 @@ import org.joda.time.Instant; import org.joda.time.MutableDateTime; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** - * Tests for splittable {@link DoFn} behavior. */ @RunWith(JUnit4.class) public class SplittableDoFnTest { @@ -155,9 +149,9 @@ public void process(ProcessContext c) { } @Test + @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testPairWithIndexBasic() { Pipeline p = TestPipeline.create(); - p.getOptions().setRunner(DirectRunner.class); PCollection> res = p.apply(Create.of("a", "bb", "ccccc")) .apply(ParDo.of(new PairStringWithIndexToLength())) @@ -179,11 +173,11 @@ public void testPairWithIndexBasic() { } @Test + @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testPairWithIndexWindowedTimestamped() { // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps // of elements in the input collection. Pipeline p = TestPipeline.create(); - p.getOptions().setRunner(DirectRunner.class); MutableDateTime mutableNow = Instant.now().toMutableDateTime(); mutableNow.setMillisOfSecond(0); @@ -267,9 +261,9 @@ public OffsetRangeTracker newTracker(OffsetRange range) { } @Test + @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testSideInputsAndOutputs() throws Exception { Pipeline p = TestPipeline.create(); - p.getOptions().setRunner(DirectRunner.class); PCollectionView sideInput = p.apply("side input", Create.of("foo")).apply(View.asSingleton()); @@ -294,9 +288,9 @@ public void testSideInputsAndOutputs() throws Exception { } @Test + @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testLateData() throws Exception { Pipeline p = TestPipeline.create(); - p.getOptions().setRunner(DirectRunner.class); Instant base = Instant.now(); @@ -389,9 +383,9 @@ public void tearDown() { } @Test + @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testLifecycleMethods() throws Exception { Pipeline p = TestPipeline.create(); - p.getOptions().setRunner(DirectRunner.class); PCollection res = p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle())); From ffa81edd0ec4d9a8150280efdb6a6de412114743 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 1 Dec 2016 21:03:04 -0800 Subject: [PATCH 057/279] Fix pom syntax for excludedGroups for SplittableParDo --- runners/apex/pom.xml | 6 ++++-- runners/flink/runner/pom.xml | 11 ++++++++--- runners/google-cloud-dataflow-java/pom.xml | 6 ++++-- runners/spark/pom.xml | 6 ++++-- 4 files changed, 20 insertions(+), 9 deletions(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 983781d3803f..629e89080803 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -185,8 +185,10 @@ org.apache.beam.sdk.testing.RunnableOnService - org.apache.beam.sdk.testing.UsesStatefulParDo - org.apache.beam.sdk.testing.UsesSplittableParDo + + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesSplittableParDo + none true diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 3e3dd7ec0ac5..615d5f1eca59 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -53,8 +53,10 @@ org.apache.beam.sdk.testing.RunnableOnService - org.apache.beam.sdk.testing.UsesStatefulParDo - org.apache.beam.sdk.testing.UsesSplittableParDo + + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesSplittableParDo + none true @@ -80,7 +82,10 @@ org.apache.beam.sdk.testing.RunnableOnService - org.apache.beam.sdk.testing.UsesStatefulParDo + + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesSplittableParDo + none true diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 85474995c1d2..adebb2a7fd8d 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -77,8 +77,10 @@ runnable-on-service-tests - org.apache.beam.sdk.testing.UsesStatefulParDo - org.apache.beam.sdk.testing.UsesSplittableParDo + + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesSplittableParDo + org.apache.beam.sdk.transforms.FlattenTest diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index dc000bfa8574..e34af15cc448 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -72,8 +72,10 @@ org.apache.beam.sdk.testing.RunnableOnService - org.apache.beam.sdk.testing.UsesStatefulParDo - org.apache.beam.sdk.testing.UsesSplittableParDo + + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesSplittableParDo + 1 false true From 8fd520c07e464c4308d8d32cc0e88e2ecd96c8d2 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 1 Dec 2016 11:21:30 -0800 Subject: [PATCH 058/279] DataflowRunner: reject job submission when the version has not been properly set --- .../org/apache/beam/runners/dataflow/DataflowRunner.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 6ed386aff0bc..0357b4650515 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -513,10 +513,14 @@ public DataflowPipelineJob run(Pipeline pipeline) { Job newJob = jobSpecification.getJob(); newJob.setClientRequestId(requestId); - String version = ReleaseInfo.getReleaseInfo().getVersion(); + ReleaseInfo releaseInfo = ReleaseInfo.getReleaseInfo(); + String version = releaseInfo.getVersion(); + checkState( + !version.equals("${pom.version}"), + "Unable to submit a job to the Dataflow service with unset version ${pom.version}"); System.out.println("Dataflow SDK version: " + version); - newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo()); + newJob.getEnvironment().setUserAgent(releaseInfo); // The Dataflow Service may write to the temporary directory directly, so // must be verified. if (!isNullOrEmpty(options.getGcpTempLocation())) { From d6eb5143b17eca9e5a59eaf6d2e3cd696e8bb38c Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 1 Dec 2016 10:04:38 -0800 Subject: [PATCH 059/279] travis.yml: disable skipping things that no longer run --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 9e1406cbae45..a806477a13d8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -30,7 +30,7 @@ notifications: env: global: - - MAVEN_OVERRIDE="--settings=.travis/settings.xml -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true" + - MAVEN_OVERRIDE="--settings=.travis/settings.xml" - MAVEN_CONTAINER_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'" matrix: From d989123424a54699ecb47ba6c0a4e437316cabce Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Mon, 31 Oct 2016 15:46:25 -0700 Subject: [PATCH 060/279] Move Towards removing WindowedValue from SDK - Introduces ValueInSingleWindow for purposes of PAssert - Uses ValueInSingleWindow inside DoFnTester - Moves WindowMatchers{,Test} to runners-core After this commit, WindowedValue does not appear in any SDK APIs used by Pipeline authors. --- .../beam/runners/core/ReduceFnRunnerTest.java | 5 +- .../runners/core/SplittableParDoTest.java | 38 ++--- .../beam/runners/core}/WindowMatchers.java | 2 +- .../runners/core}/WindowMatchersTest.java | 2 +- .../direct/WindowEvaluatorFactoryTest.java | 4 +- .../sdk/{util => testing}/GatherAllPanes.java | 24 ++-- .../org/apache/beam/sdk/testing/PAssert.java | 77 +++++----- .../beam/sdk/testing/PaneExtractors.java | 55 ++++--- .../beam/sdk/testing/ValueInSingleWindow.java | 134 ++++++++++++++++++ .../beam/sdk/transforms/DoFnTester.java | 58 ++++---- .../beam/sdk/util/IdentityWindowFn.java | 2 +- .../{util => testing}/GatherAllPanesTest.java | 27 ++-- .../beam/sdk/testing/PaneExtractorsTest.java | 133 ++++++++--------- .../testing/ValueInSingleWindowCoderTest.java | 51 +++++++ 14 files changed, 406 insertions(+), 206 deletions(-) rename {sdks/java/core/src/test/java/org/apache/beam/sdk => runners/core-java/src/test/java/org/apache/beam/runners/core}/WindowMatchers.java (99%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk => runners/core-java/src/test/java/org/apache/beam/runners/core}/WindowMatchersTest.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => testing}/GatherAllPanes.java (80%) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java rename sdks/java/core/src/test/java/org/apache/beam/sdk/{util => testing}/GatherAllPanesTest.java (84%) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 20eb08b67ecd..ba5756798ee5 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; -import static org.apache.beam.sdk.WindowMatchers.isWindowedValue; +import static org.apache.beam.runners.core.WindowMatchers.isSingleWindowedValue; +import static org.apache.beam.runners.core.WindowMatchers.isWindowedValue; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; @@ -39,7 +39,6 @@ import java.util.Iterator; import java.util.List; import org.apache.beam.runners.core.triggers.TriggerStateMachine; -import org.apache.beam.sdk.WindowMatchers; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index 990d8922e311..b13d83956e1a 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.ValueInSingleWindow; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; @@ -142,14 +143,15 @@ public void testBoundednessForBoundedFn() { PCollection.IsBounded.BOUNDED, makeBoundedCollection(pipeline) .apply("bounded to bounded", new SplittableParDo<>(makeParDo(boundedFn))) - .get(MAIN_OUTPUT_TAG).isBounded()); + .get(MAIN_OUTPUT_TAG) + .isBounded()); assertEquals( "Applying a bounded SDF to an unbounded collection produces an unbounded collection", PCollection.IsBounded.UNBOUNDED, makeUnboundedCollection(pipeline) - .apply( - "bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn))) - .get(MAIN_OUTPUT_TAG).isBounded()); + .apply("bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); } @Test @@ -160,18 +162,16 @@ public void testBoundednessForUnboundedFn() { "Applying an unbounded SDF to a bounded collection produces a bounded collection", PCollection.IsBounded.UNBOUNDED, makeBoundedCollection(pipeline) - .apply( - "unbounded to bounded", - new SplittableParDo<>(makeParDo(unboundedFn))) - .get(MAIN_OUTPUT_TAG).isBounded()); + .apply("unbounded to bounded", new SplittableParDo<>(makeParDo(unboundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); assertEquals( "Applying an unbounded SDF to an unbounded collection produces an unbounded collection", PCollection.IsBounded.UNBOUNDED, makeUnboundedCollection(pipeline) - .apply( - "unbounded to unbounded", - new SplittableParDo<>(makeParDo(unboundedFn))) - .get(MAIN_OUTPUT_TAG).isBounded()); + .apply("unbounded to unbounded", new SplittableParDo<>(makeParDo(unboundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); } // ------------------------------- Tests for ProcessFn --------------------------------- @@ -224,9 +224,11 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - tester - .getMutableOutput(tester.getMainOutputTag()) - .add(WindowedValue.of(output, timestamp, windows, pane)); + for (BoundedWindow window : windows) { + tester + .getMutableOutput(tester.getMainOutputTag()) + .add(ValueInSingleWindow.of(output, timestamp, window, pane)); + } } @Override @@ -236,7 +238,11 @@ public void sideOutputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - tester.getMutableOutput(tag).add(WindowedValue.of(output, timestamp, windows, pane)); + for (BoundedWindow window : windows) { + tester + .getMutableOutput(tag) + .add(ValueInSingleWindow.of(output, timestamp, window, pane)); + } } }); // Do not clone since ProcessFn references non-serializable DoFnTester itself diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java similarity index 99% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java index 3531a8631cbe..6c3a7e22290d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk; +package org.apache.beam.runners.core; import com.google.common.collect.Lists; import java.util.Collection; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java similarity index 98% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java index 89637e2af270..6f4741a7d54b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk; +package org.apache.beam.runners.core; import static org.junit.Assert.assertThat; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index e2f987c73c13..66c28ce612a2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.direct; -import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; -import static org.apache.beam.sdk.WindowMatchers.isWindowedValue; +import static org.apache.beam.runners.core.WindowMatchers.isSingleWindowedValue; +import static org.apache.beam.runners.core.WindowMatchers.isWindowedValue; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java similarity index 80% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java index 52a2ba86416f..2b311b79a715 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.testing; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; @@ -27,6 +27,7 @@ import org.apache.beam.sdk.transforms.windowing.Never; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.IdentityWindowFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; @@ -38,8 +39,8 @@ * watermark passes the end of the window plus allowed lateness) even if the upstream triggers * closed the window earlier. */ -public class GatherAllPanes - extends PTransform, PCollection>>> { +class GatherAllPanes + extends PTransform, PCollection>>> { /** * Gathers all panes of each window into a single output element. * @@ -54,33 +55,34 @@ public static GatherAllPanes globally() { private GatherAllPanes() {} @Override - public PCollection>> apply(PCollection input) { + public PCollection>> apply(PCollection input) { WindowFn originalWindowFn = input.getWindowingStrategy().getWindowFn(); return input .apply(ParDo.of(new ReifyTimestampsAndWindowsFn())) .setCoder( - WindowedValue.FullWindowedValueCoder.of( + ValueInSingleWindow.Coder.of( input.getCoder(), input.getWindowingStrategy().getWindowFn().windowCoder())) .apply( - WithKeys.>of(0).withKeyType(new TypeDescriptor() {})) + WithKeys.>of(0) + .withKeyType(new TypeDescriptor() {})) .apply( Window.into( - new IdentityWindowFn>>( + new IdentityWindowFn>>( originalWindowFn.windowCoder())) .triggering(Never.ever()) .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness()) .discardingFiredPanes()) // all values have the same key so they all appear as a single output element - .apply(GroupByKey.>create()) - .apply(Values.>>create()) + .apply(GroupByKey.>create()) + .apply(Values.>>create()) .setWindowingStrategyInternal(input.getWindowingStrategy()); } - private static class ReifyTimestampsAndWindowsFn extends DoFn> { + private static class ReifyTimestampsAndWindowsFn extends DoFn> { @DoFn.ProcessElement public void processElement(ProcessContext c, BoundedWindow window) { - c.output(WindowedValue.of(c.element(), c.timestamp(), window, c.pane())); + c.output(ValueInSingleWindow.of(c.element(), c.timestamp(), window, c.pane())); } } } 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 b3a14aa6559b..7dc78d86d61c 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 @@ -63,8 +63,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.GatherAllPanes; -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.PBegin; @@ -349,7 +347,7 @@ public static SingletonAssert> thatMap(PCollection> ac private static class PCollectionContentsAssert implements IterableAssert { private final PCollection actual; private final AssertionWindows rewindowingStrategy; - private final SimpleFunction>, Iterable> paneExtractor; + private final SimpleFunction>, Iterable> paneExtractor; public PCollectionContentsAssert(PCollection actual) { this(actual, IntoGlobalWindow.of(), PaneExtractors.allPanes()); @@ -358,7 +356,7 @@ public PCollectionContentsAssert(PCollection actual) { public PCollectionContentsAssert( PCollection actual, AssertionWindows rewindowingStrategy, - SimpleFunction>, Iterable> paneExtractor) { + SimpleFunction>, Iterable> paneExtractor) { this.actual = actual; this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; @@ -391,7 +389,7 @@ public IterableAssert inEarlyGlobalWindowPanes() { private PCollectionContentsAssert withPane( BoundedWindow window, - SimpleFunction>, Iterable> paneExtractor) { + SimpleFunction>, Iterable> paneExtractor) { @SuppressWarnings({"unchecked", "rawtypes"}) Coder windowCoder = (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(); @@ -523,7 +521,7 @@ private static class PCollectionSingletonIterableAssert implements IterableAs private final PCollection> actual; private final Coder elementCoder; private final AssertionWindows rewindowingStrategy; - private final SimpleFunction>>, Iterable>> + private final SimpleFunction>>, Iterable>> paneExtractor; public PCollectionSingletonIterableAssert(PCollection> actual) { @@ -533,7 +531,8 @@ public PCollectionSingletonIterableAssert(PCollection> actual) { public PCollectionSingletonIterableAssert( PCollection> actual, AssertionWindows rewindowingStrategy, - SimpleFunction>>, Iterable>> paneExtractor) { + SimpleFunction>>, Iterable>> + paneExtractor) { this.actual = actual; @SuppressWarnings("unchecked") @@ -571,7 +570,8 @@ public IterableAssert inEarlyGlobalWindowPanes() { private PCollectionSingletonIterableAssert withPanes( BoundedWindow window, - SimpleFunction>>, Iterable>> paneExtractor) { + SimpleFunction>>, Iterable>> + paneExtractor) { @SuppressWarnings({"unchecked", "rawtypes"}) Coder windowCoder = (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(); @@ -620,7 +620,8 @@ private static class PCollectionViewAssert implements SingletonAss private final PCollection actual; private final PTransform, PCollectionView> view; private final AssertionWindows rewindowActuals; - private final SimpleFunction>, Iterable> paneExtractor; + private final SimpleFunction>, Iterable> + paneExtractor; private final Coder coder; protected PCollectionViewAssert( @@ -634,7 +635,7 @@ private PCollectionViewAssert( PCollection actual, PTransform, PCollectionView> view, AssertionWindows rewindowActuals, - SimpleFunction>, Iterable> paneExtractor, + SimpleFunction>, Iterable> paneExtractor, Coder coder) { this.actual = actual; this.view = view; @@ -660,7 +661,7 @@ public PCollectionViewAssert inOnTimePane(BoundedWindow window) { private PCollectionViewAssert inPane( BoundedWindow window, - SimpleFunction>, Iterable> paneExtractor) { + SimpleFunction>, Iterable> paneExtractor) { return new PCollectionViewAssert<>( actual, view, @@ -738,13 +739,14 @@ private static class CreateActual private final transient PCollection actual; private final transient AssertionWindows rewindowActuals; - private final transient SimpleFunction>, Iterable> extractPane; + private final transient SimpleFunction>, Iterable> + extractPane; private final transient PTransform, PCollectionView> actualView; public static CreateActual from( PCollection actual, AssertionWindows rewindowActuals, - SimpleFunction>, Iterable> extractPane, + SimpleFunction>, Iterable> extractPane, PTransform, PCollectionView> actualView) { return new CreateActual<>(actual, rewindowActuals, extractPane, actualView); } @@ -752,7 +754,7 @@ public static CreateActual from( private CreateActual( PCollection actual, AssertionWindows rewindowActuals, - SimpleFunction>, Iterable> extractPane, + SimpleFunction>, Iterable> extractPane, PTransform, PCollectionView> actualView) { this.actual = actual; this.rewindowActuals = rewindowActuals; @@ -822,7 +824,7 @@ public Void apply(T actual) { * a single empty iterable, even though in practice most runners will not produce any element. */ private static class GroupGlobally - extends PTransform, PCollection>>> + extends PTransform, PCollection>>> implements Serializable { private final AssertionWindows rewindowingStrategy; @@ -831,20 +833,20 @@ public GroupGlobally(AssertionWindows rewindowingStrategy) { } @Override - public PCollection>> apply(PCollection input) { + public PCollection>> apply(PCollection input) { final int combinedKey = 42; // Remove the triggering on both PTransform< - PCollection>>>, - PCollection>>>> + PCollection>>>, + PCollection>>>> removeTriggering = - Window.>>>triggering(Never.ever()) + Window.>>>triggering(Never.ever()) .discardingFiredPanes() .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness()); // Group the contents by key. If it is empty, this PCollection will be empty, too. // Then key it again with a dummy key. - PCollection>>> groupedContents = + PCollection>>> groupedContents = // TODO: Split the filtering from the rewindowing, and apply filtering before the Gather // if the grouping of extra records input @@ -852,45 +854,47 @@ public PCollection>> apply(PCollection input) { .apply("GatherAllOutputs", GatherAllPanes.globally()) .apply( "RewindowActuals", - rewindowingStrategy.>>windowActuals()) - .apply("KeyForDummy", WithKeys.>>of(combinedKey)) + rewindowingStrategy.>>windowActuals()) + .apply( + "KeyForDummy", + WithKeys.>>of(combinedKey)) .apply("RemoveActualsTriggering", removeTriggering); // Create another non-empty PCollection that is keyed with a distinct dummy key - PCollection>>> keyedDummy = + PCollection>>> keyedDummy = input .getPipeline() .apply( Create.of( KV.of( combinedKey, - (Iterable>) - Collections.>emptyList())) + (Iterable>) + Collections.>emptyList())) .withCoder(groupedContents.getCoder())) .apply( "WindowIntoDummy", - rewindowingStrategy.>>>windowDummy()) + rewindowingStrategy.>>>windowDummy()) .apply("RemoveDummyTriggering", removeTriggering); // Flatten them together and group by the combined key to get a single element - PCollection>>>> dummyAndContents = + PCollection>>>> dummyAndContents = PCollectionList.of(groupedContents) .and(keyedDummy) .apply( "FlattenDummyAndContents", - Flatten.>>>pCollections()) + Flatten.>>>pCollections()) .apply( "NeverTrigger", - Window.>>>triggering(Never.ever()) + Window.>>>triggering(Never.ever()) .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness()) .discardingFiredPanes()) .apply( "GroupDummyAndContents", - GroupByKey.>>create()); + GroupByKey.>>create()); return dummyAndContents - .apply(Values.>>>create()) - .apply(ParDo.of(new ConcatFn>())); + .apply(Values.>>>create()) + .apply(ParDo.of(new ConcatFn>())); } } @@ -909,12 +913,12 @@ public static class GroupThenAssert extends PTransform, PDone> implements Serializable { private final SerializableFunction, Void> checkerFn; private final AssertionWindows rewindowingStrategy; - private final SimpleFunction>, Iterable> paneExtractor; + private final SimpleFunction>, Iterable> paneExtractor; private GroupThenAssert( SerializableFunction, Void> checkerFn, AssertionWindows rewindowingStrategy, - SimpleFunction>, Iterable> paneExtractor) { + SimpleFunction>, Iterable> paneExtractor) { this.checkerFn = checkerFn; this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; @@ -940,13 +944,14 @@ public static class GroupThenAssertForSingleton extends PTransform>, PDone> implements Serializable { private final SerializableFunction, Void> checkerFn; private final AssertionWindows rewindowingStrategy; - private final SimpleFunction>>, Iterable>> + private final SimpleFunction>>, Iterable>> paneExtractor; private GroupThenAssertForSingleton( SerializableFunction, Void> checkerFn, AssertionWindows rewindowingStrategy, - SimpleFunction>>, Iterable>> paneExtractor) { + SimpleFunction>>, Iterable>> + paneExtractor) { this.checkerFn = checkerFn; this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java index db72a0cd4c22..dd1fac9d5546 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java @@ -25,14 +25,13 @@ import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; /** - * {@link PTransform PTransforms} which take an {@link Iterable} of {@link WindowedValue - * WindowedValues} and outputs an {@link Iterable} of all values in the specified pane, dropping the - * {@link WindowedValue} metadata. + * {@link PTransform PTransforms} which take an {@link Iterable} of {@link ValueInSingleWindow + * ValueInSingleWindows} and outputs an {@link Iterable} of all values in the specified pane, + * dropping the {@link ValueInSingleWindow} metadata. * *

    Although all of the method signatures return SimpleFunction, users should ensure to set the * coder of any output {@link PCollection}, as appropriate {@link TypeDescriptor TypeDescriptors} @@ -42,36 +41,36 @@ final class PaneExtractors { private PaneExtractors() { } - static SimpleFunction>, Iterable> onlyPane() { + static SimpleFunction>, Iterable> onlyPane() { return new ExtractOnlyPane<>(); } - static SimpleFunction>, Iterable> onTimePane() { + static SimpleFunction>, Iterable> onTimePane() { return new ExtractOnTimePane<>(); } - static SimpleFunction>, Iterable> finalPane() { + static SimpleFunction>, Iterable> finalPane() { return new ExtractFinalPane<>(); } - static SimpleFunction>, Iterable> nonLatePanes() { + static SimpleFunction>, Iterable> nonLatePanes() { return new ExtractNonLatePanes<>(); } - static SimpleFunction>, Iterable> earlyPanes() { + static SimpleFunction>, Iterable> earlyPanes() { return new ExtractEarlyPanes<>(); } - static SimpleFunction>, Iterable> allPanes() { + static SimpleFunction>, Iterable> allPanes() { return new ExtractAllPanes<>(); } private static class ExtractOnlyPane - extends SimpleFunction>, Iterable> { + extends SimpleFunction>, Iterable> { @Override - public Iterable apply(Iterable> input) { + public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); - for (WindowedValue value : input) { + for (ValueInSingleWindow value : input) { checkState(value.getPane().isFirst() && value.getPane().isLast(), "Expected elements to be produced by a trigger that fires at most once, but got" + "a value in a pane that is %s. Actual Pane Info: %s", @@ -85,11 +84,11 @@ public Iterable apply(Iterable> input) { private static class ExtractOnTimePane - extends SimpleFunction>, Iterable> { + extends SimpleFunction>, Iterable> { @Override - public Iterable apply(Iterable> input) { + public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); - for (WindowedValue value : input) { + for (ValueInSingleWindow value : input) { if (value.getPane().getTiming().equals(Timing.ON_TIME)) { outputs.add(value.getValue()); } @@ -100,11 +99,11 @@ public Iterable apply(Iterable> input) { private static class ExtractFinalPane - extends SimpleFunction>, Iterable> { + extends SimpleFunction>, Iterable> { @Override - public Iterable apply(Iterable> input) { + public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); - for (WindowedValue value : input) { + for (ValueInSingleWindow value : input) { if (value.getPane().isLast()) { outputs.add(value.getValue()); } @@ -115,11 +114,11 @@ public Iterable apply(Iterable> input) { private static class ExtractAllPanes - extends SimpleFunction>, Iterable> { + extends SimpleFunction>, Iterable> { @Override - public Iterable apply(Iterable> input) { + public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); - for (WindowedValue value : input) { + for (ValueInSingleWindow value : input) { outputs.add(value.getValue()); } return outputs; @@ -128,11 +127,11 @@ public Iterable apply(Iterable> input) { private static class ExtractNonLatePanes - extends SimpleFunction>, Iterable> { + extends SimpleFunction>, Iterable> { @Override - public Iterable apply(Iterable> input) { + public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); - for (WindowedValue value : input) { + for (ValueInSingleWindow value : input) { if (value.getPane().getTiming() != PaneInfo.Timing.LATE) { outputs.add(value.getValue()); } @@ -142,11 +141,11 @@ public Iterable apply(Iterable> input) { } private static class ExtractEarlyPanes - extends SimpleFunction>, Iterable> { + extends SimpleFunction>, Iterable> { @Override - public Iterable apply(Iterable> input) { + public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); - for (WindowedValue value : input) { + for (ValueInSingleWindow value : input) { if (value.getPane().getTiming() == PaneInfo.Timing.EARLY) { outputs.add(value.getValue()); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java new file mode 100644 index 000000000000..9ec030ff96b2 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PropertyNames; +import org.joda.time.Instant; + +/** + * An immutable tuple of value, timestamp, window, and pane. + * + * @param the type of the value + */ +@AutoValue +public abstract class ValueInSingleWindow { + /** Returns the value of this {@code ValueInSingleWindow}. */ + @Nullable + public abstract T getValue(); + + /** Returns the timestamp of this {@code ValueInSingleWindow}. */ + public abstract Instant getTimestamp(); + + /** Returns the window of this {@code ValueInSingleWindow}. */ + public abstract BoundedWindow getWindow(); + + /** Returns the pane of this {@code ValueInSingleWindow} in its window. */ + public abstract PaneInfo getPane(); + + public static ValueInSingleWindow of( + T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) { + return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, paneInfo); + } + + /** A coder for {@link ValueInSingleWindow}. */ + public static class Coder extends StandardCoder> { + private final org.apache.beam.sdk.coders.Coder valueCoder; + private final org.apache.beam.sdk.coders.Coder windowCoder; + + public static Coder of( + org.apache.beam.sdk.coders.Coder valueCoder, + org.apache.beam.sdk.coders.Coder windowCoder) { + return new Coder<>(valueCoder, windowCoder); + } + + @JsonCreator + public static Coder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + checkArgument(components.size() == 2, "Expecting 2 components, got %s", components.size()); + @SuppressWarnings("unchecked") + org.apache.beam.sdk.coders.Coder valueCoder = + (org.apache.beam.sdk.coders.Coder) components.get(0); + @SuppressWarnings("unchecked") + org.apache.beam.sdk.coders.Coder windowCoder = + (org.apache.beam.sdk.coders.Coder) components.get(1); + return new Coder<>(valueCoder, windowCoder); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + Coder( + org.apache.beam.sdk.coders.Coder valueCoder, + org.apache.beam.sdk.coders.Coder windowCoder) { + this.valueCoder = valueCoder; + this.windowCoder = (org.apache.beam.sdk.coders.Coder) windowCoder; + } + + @Override + public void encode(ValueInSingleWindow windowedElem, OutputStream outStream, Context context) + throws IOException { + Context nestedContext = context.nested(); + valueCoder.encode(windowedElem.getValue(), outStream, nestedContext); + InstantCoder.of().encode(windowedElem.getTimestamp(), outStream, nestedContext); + windowCoder.encode(windowedElem.getWindow(), outStream, nestedContext); + PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream, context); + } + + @Override + public ValueInSingleWindow decode(InputStream inStream, Context context) throws IOException { + Context nestedContext = context.nested(); + T value = valueCoder.decode(inStream, nestedContext); + Instant timestamp = InstantCoder.of().decode(inStream, nestedContext); + BoundedWindow window = windowCoder.decode(inStream, nestedContext); + PaneInfo pane = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream, nestedContext); + return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, pane); + } + + @Override + public List> getCoderArguments() { + // Coder arguments are coders for the type parameters of the coder - i.e. only T. + return ImmutableList.of(valueCoder); + } + + @Override + public List> getComponents() { + // Coder components are all inner coders that it uses - i.e. both T and BoundedWindow. + return ImmutableList.of(valueCoder, windowCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + valueCoder.verifyDeterministic(); + windowCoder.verifyDeterministic(); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 0c6043fc310d..17fa612aedb7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -23,7 +23,6 @@ import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Arrays; @@ -35,8 +34,10 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.ValueInSingleWindow; import org.apache.beam.sdk.transforms.Combine.CombineFn; 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.SerializableUtils; import org.apache.beam.sdk.util.TimerInternals; @@ -353,10 +354,10 @@ public OutputT apply(TimestampedValue input) { public List> peekOutputElementsWithTimestamp() { // TODO: Should we return an unmodifiable list? return Lists.transform(getImmutableOutput(mainOutputTag), - new Function, TimestampedValue>() { + new Function, TimestampedValue>() { @Override @SuppressWarnings("unchecked") - public TimestampedValue apply(WindowedValue input) { + public TimestampedValue apply(ValueInSingleWindow input) { return TimestampedValue.of(input.getValue(), input.getTimestamp()); } }); @@ -378,8 +379,8 @@ public List> peekOutputElementsInWindow( TupleTag tag, BoundedWindow window) { ImmutableList.Builder> valuesBuilder = ImmutableList.builder(); - for (WindowedValue value : getImmutableOutput(tag)) { - if (value.getWindows().contains(window)) { + for (ValueInSingleWindow value : getImmutableOutput(tag)) { + if (value.getWindow().equals(window)) { valuesBuilder.add(TimestampedValue.of(value.getValue(), value.getTimestamp())); } } @@ -434,10 +435,10 @@ public List> takeOutputElementsWithTimestamp() { public List peekSideOutputElements(TupleTag tag) { // TODO: Should we return an unmodifiable list? return Lists.transform(getImmutableOutput(tag), - new Function, T>() { + new Function, T>() { @SuppressWarnings("unchecked") @Override - public T apply(WindowedValue input) { + public T apply(ValueInSingleWindow input) { return input.getValue(); }}); } @@ -510,16 +511,16 @@ private AggregateT extractAggregatorValue( return combiner.extractOutput(accumulator); } - private List> getImmutableOutput(TupleTag tag) { + private List> getImmutableOutput(TupleTag tag) { @SuppressWarnings({"unchecked", "rawtypes"}) - List> elems = (List) outputs.get(tag); + List> elems = (List) outputs.get(tag); return ImmutableList.copyOf( - MoreObjects.firstNonNull(elems, Collections.>emptyList())); + MoreObjects.firstNonNull(elems, Collections.>emptyList())); } @SuppressWarnings({"unchecked", "rawtypes"}) - public List> getMutableOutput(TupleTag tag) { - List> outputList = (List) outputs.get(tag); + public List> getMutableOutput(TupleTag tag) { + List> outputList = (List) outputs.get(tag); if (outputList == null) { outputList = new ArrayList<>(); outputs.put(tag, (List) outputList); @@ -612,23 +613,22 @@ public void sideOutput(TupleTag tag, T output) { sideOutputWithTimestamp(tag, output, BoundedWindow.TIMESTAMP_MIN_VALUE); } - public void noteOutput(TupleTag tag, WindowedValue output) { + public void noteOutput(TupleTag tag, ValueInSingleWindow output) { getMutableOutput(tag).add(output); } } private TestProcessContext createProcessContext(TimestampedValue elem) { - WindowedValue windowedValue = WindowedValue.timestampedValueInGlobalWindow( - elem.getValue(), elem.getTimestamp()); - - return new TestProcessContext(windowedValue); + return new TestProcessContext( + ValueInSingleWindow.of( + elem.getValue(), elem.getTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } private class TestProcessContext extends OldDoFn.ProcessContext { private final TestContext context; - private final WindowedValue element; + private final ValueInSingleWindow element; - private TestProcessContext(WindowedValue element) { + private TestProcessContext(ValueInSingleWindow element) { fn.super(); this.context = createContext(fn); this.element = element; @@ -661,7 +661,7 @@ public Instant timestamp() { @Override public BoundedWindow window() { - return Iterables.getOnlyElement(element.getWindows()); + return element.getWindow(); } @Override @@ -683,7 +683,10 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - context.noteOutput(mainOutputTag, WindowedValue.of(output, timestamp, windows, pane)); + for (BoundedWindow window : windows) { + context.noteOutput( + mainOutputTag, ValueInSingleWindow.of(output, timestamp, window, pane)); + } } @Override @@ -693,7 +696,10 @@ public void sideOutputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - context.noteOutput(tag, WindowedValue.of(output, timestamp, windows, pane)); + for (BoundedWindow window : windows) { + context.noteOutput( + tag, ValueInSingleWindow.of(output, timestamp, window, pane)); + } } @Override @@ -703,7 +709,7 @@ public TimerInternals timerInternals() { @Override public Collection windows() { - return element.getWindows(); + return Collections.singleton(element.getWindow()); } @Override @@ -742,8 +748,8 @@ public void sideOutput(TupleTag tag, T output) { @Override public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.noteOutput(tag, - WindowedValue.of(output, timestamp, element.getWindows(), element.getPane())); + context.noteOutput( + tag, ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane())); } @Override @@ -803,7 +809,7 @@ private enum State { OldDoFn fn; /** The outputs from the {@link DoFn} under test. */ - private Map, List>> outputs; + private Map, List>> outputs; private InMemoryStateInternals stateInternals; private InMemoryTimerInternals timerInternals; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java index 8ca1bfd77b5b..c02e1f44b2a2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java @@ -45,7 +45,7 @@ *

    This {@link WindowFn} is an internal implementation detail of sdk-provided utilities, and * should not be used by {@link Pipeline} writers. */ -class IdentityWindowFn extends NonMergingWindowFn { +public class IdentityWindowFn extends NonMergingWindowFn { /** * The coder of the type of windows of the input {@link PCollection}. This is not an arbitrary diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java similarity index 84% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java index d195623d4634..417147f88fbd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java @@ -15,16 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.testing; import static org.junit.Assert.fail; import com.google.common.collect.Iterables; import java.io.Serializable; import org.apache.beam.sdk.io.CountingInput; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -45,16 +42,14 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for {@link GatherAllPanes}. - */ +/** Tests for {@link GatherAllPanes}. */ @RunWith(JUnit4.class) public class GatherAllPanesTest implements Serializable { @Test @Category(NeedsRunner.class) public void singlePaneSingleReifiedPane() { TestPipeline p = TestPipeline.create(); - PCollection>>> accumulatedPanes = + PCollection>>> accumulatedPanes = p.apply(CountingInput.upTo(20000)) .apply( WithTimestamps.of( @@ -76,10 +71,11 @@ public Instant apply(Long input) { PAssert.that(accumulatedPanes) .satisfies( - new SerializableFunction>>>, Void>() { + new SerializableFunction< + Iterable>>>, Void>() { @Override - public Void apply(Iterable>>> input) { - for (Iterable>> windowedInput : input) { + public Void apply(Iterable>>> input) { + for (Iterable>> windowedInput : input) { if (Iterables.size(windowedInput) > 1) { fail("Expected all windows to have exactly one pane, got " + windowedInput); return null; @@ -99,7 +95,7 @@ public void multiplePanesMultipleReifiedPane() { PCollection someElems = p.apply("someLongs", CountingInput.upTo(20000)); PCollection otherElems = p.apply("otherLongs", CountingInput.upTo(20000)); - PCollection>>> accumulatedPanes = + PCollection>>> accumulatedPanes = PCollectionList.of(someElems) .and(otherElems) .apply(Flatten.pCollections()) @@ -125,10 +121,11 @@ public Instant apply(Long input) { PAssert.that(accumulatedPanes) .satisfies( - new SerializableFunction>>>, Void>() { + new SerializableFunction< + Iterable>>>, Void>() { @Override - public Void apply(Iterable>>> input) { - for (Iterable>> windowedInput : input) { + public Void apply(Iterable>>> input) { + for (Iterable>> windowedInput : input) { if (Iterables.size(windowedInput) > 1) { return null; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java index ef501d495c50..7df2f893fb59 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java @@ -23,10 +23,10 @@ import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.SerializableFunction; +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.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -34,32 +34,33 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for {@link PaneExtractors}. - */ +/** Tests for {@link PaneExtractors}. */ @RunWith(JUnit4.class) public class PaneExtractorsTest { @Rule public ExpectedException thrown = ExpectedException.none(); @Test public void onlyPaneNoFiring() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.onlyPane(); - Iterable> noFiring = + Iterable> noFiring = ImmutableList.of( - WindowedValue.valueInGlobalWindow(9), WindowedValue.valueInEmptyWindows(19)); + ValueInSingleWindow.of( + 9, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + ValueInSingleWindow.of( + 19, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); assertThat(extractor.apply(noFiring), containsInAnyOrder(9, 19)); } @Test public void onlyPaneOnlyOneFiring() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.onlyPane(); - Iterable> onlyFiring = + Iterable> onlyFiring = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyFiring), containsInAnyOrder(2, 1)); @@ -67,21 +68,21 @@ public void onlyPaneOnlyOneFiring() { @Test public void onlyPaneMultiplePanesFails() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.onlyPane(); - Iterable> multipleFiring = + Iterable> multipleFiring = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY)), - WindowedValue.of( + ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, @@ -94,16 +95,16 @@ public void onlyPaneMultiplePanesFails() { @Test public void onTimePane() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.onTimePane(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, @@ -114,26 +115,26 @@ public void onTimePane() { @Test public void onTimePaneOnlyEarlyAndLate() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.onTimePane(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, @@ -144,21 +145,21 @@ public void onTimePaneOnlyEarlyAndLate() { @Test public void finalPane() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.finalPane(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, true, Timing.LATE, 2L, 1L)), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, @@ -169,21 +170,21 @@ public void finalPane() { @Test public void finalPaneNoExplicitFinalEmpty() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.finalPane(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, @@ -194,15 +195,15 @@ public void finalPaneNoExplicitFinalEmpty() { @Test public void nonLatePanesSingleOnTime() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.nonLatePanes(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8)); @@ -210,16 +211,16 @@ public void nonLatePanesSingleOnTime() { @Test public void nonLatePanesSingleEarly() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.nonLatePanes(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY)), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, @@ -230,11 +231,11 @@ public void nonLatePanesSingleEarly() { @Test public void allPanesSingleLate() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.nonLatePanes(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, @@ -245,22 +246,22 @@ public void allPanesSingleLate() { @Test public void nonLatePanesMultiplePanes() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.nonLatePanes(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), - WindowedValue.of(7, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + ValueInSingleWindow.of(7, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, @@ -271,15 +272,15 @@ public void nonLatePanesMultiplePanes() { @Test public void allPanesSinglePane() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.allPanes(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8)); @@ -287,21 +288,21 @@ public void allPanesSinglePane() { @Test public void allPanesMultiplePanes() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.allPanes(); - Iterable> onlyOnTime = + Iterable> onlyOnTime = ImmutableList.of( - WindowedValue.of( + ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), - WindowedValue.of( + ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), - WindowedValue.of( + ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, @@ -312,9 +313,9 @@ public void allPanesMultiplePanes() { @Test public void allPanesEmpty() { - SerializableFunction>, Iterable> extractor = + SerializableFunction>, Iterable> extractor = PaneExtractors.allPanes(); - Iterable> noPanes = ImmutableList.of(); + Iterable> noPanes = ImmutableList.of(); assertThat(extractor.apply(noPanes), emptyIterable()); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java new file mode 100644 index 000000000000..daf73b65d4bc --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.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.sdk.testing; + +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link ValueInSingleWindow.Coder}. */ +@RunWith(JUnit4.class) +public class ValueInSingleWindowCoderTest { + @Test + public void testDecodeEncodeEqual() throws Exception { + Instant now = Instant.now(); + ValueInSingleWindow value = + ValueInSingleWindow.of( + "foo", + now, + new IntervalWindow(now, now.plus(Duration.standardSeconds(10))), + PaneInfo.NO_FIRING); + + CoderProperties.coderDecodeEncodeEqual( + ValueInSingleWindow.Coder.of(StringUtf8Coder.of(), IntervalWindow.getCoder()), value); + } + + @Test + public void testCoderSerializable() throws Exception { + CoderProperties.coderSerializable( + ValueInSingleWindow.Coder.of(StringUtf8Coder.of(), IntervalWindow.getCoder())); + } +} From 840fb3b9030908ef50937cc2e4498a2cdcb7b680 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 23 Nov 2016 14:30:57 -0800 Subject: [PATCH 061/279] Add timerId to TimerData This timerId is generated to be identical to historical behavior, and to be unique per time domain and timestamp. --- .../operators/ApexGroupByKeyOperator.java | 4 +- .../runners/direct/DirectTimerInternals.java | 4 +- .../apache/beam/sdk/util/TimerInternals.java | 107 +++++++----------- .../util/state/InMemoryTimerInternals.java | 8 +- .../beam/sdk/util/TimerInternalsTest.java | 4 +- 5 files changed, 49 insertions(+), 78 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index 3b0e4f283341..f49c7852eef7 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -426,8 +426,8 @@ protected Aggregator createAggreg public class ApexTimerInternals implements TimerInternals { @Override - public void setTimer(TimerData timerKey) { - registerActiveTimer(context.element().key(), timerKey); + public void setTimer(TimerData timerData) { + registerActiveTimer(context.element().key(), timerData); } @Override diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index 4245a87a3169..8970b4b88b8d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -53,8 +53,8 @@ public void setTimer(StateNamespace namespace, String timerId, Instant target, } @Override - public void setTimer(TimerData timerKey) { - timerUpdateBuilder.setTimer(timerKey); + public void setTimer(TimerData timerData) { + timerUpdateBuilder.setTimer(timerData); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java index 5d4a72d34853..c3e498e03ae6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java @@ -18,18 +18,16 @@ package org.apache.beam.sdk.util; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.MoreObjects; +import com.google.auto.value.AutoValue; import com.google.common.collect.ComparisonChain; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.List; -import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -50,7 +48,7 @@ public interface TimerInternals { /** - * Writes out a timer to be fired when the current time in the specified time domain reaches the + * Sets a timer to be fired when the current time in the specified time domain reaches the * target timestamp. * *

    The combination of {@code namespace} and {@code timerId} uniquely identify a timer. @@ -63,14 +61,9 @@ public interface TimerInternals { void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain); /** - * Writes out a timer to be fired when the watermark reaches the given timestamp, automatically - * generating an id for it from the provided {@link TimerData}. - * - *

    The {@link TimerData} contains all the fields necessary to set the timer. The timer's ID - * is determinstically generated from the {@link TimerData}, so it may be canceled using - * the same {@link TimerData}. + * Sets the timer described by {@code timerData}. */ - void setTimer(TimerData timerKey); + void setTimer(TimerData timerData); /** * Deletes the given timer. @@ -78,7 +71,7 @@ public interface TimerInternals { void deleteTimer(StateNamespace namespace, String timerId); /** - * Deletes the given timer, automatically inferring its ID from the {@link TimerData}. + * Deletes the timer with the ID contained in the provided {@link TimerData}. */ void deleteTimer(TimerData timerKey); @@ -163,64 +156,38 @@ public interface TimerInternals { /** * Data about a timer as represented within {@link TimerInternals}. */ - class TimerData implements Comparable { - private final StateNamespace namespace; - private final Instant timestamp; - private final TimeDomain domain; + @AutoValue + abstract class TimerData implements Comparable { - private TimerData(StateNamespace namespace, Instant timestamp, TimeDomain domain) { - this.namespace = checkNotNull(namespace); - this.timestamp = checkNotNull(timestamp); - this.domain = checkNotNull(domain); - } + public abstract String getTimerId(); - public StateNamespace getNamespace() { - return namespace; - } + public abstract StateNamespace getNamespace(); - public Instant getTimestamp() { - return timestamp; - } + public abstract Instant getTimestamp(); - public TimeDomain getDomain() { - return domain; - } + public abstract TimeDomain getDomain(); /** - * Construct the {@code TimerKey} for the given parameters. + * Construct a {@link TimerData} for the given parameters, where the timer ID is automatically + * generated. */ - public static TimerData of(StateNamespace namespace, Instant timestamp, TimeDomain domain) { - return new TimerData(namespace, timestamp, domain); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (!(obj instanceof TimerData)) { - return false; - } - - TimerData that = (TimerData) obj; - return Objects.equals(this.domain, that.domain) - && this.timestamp.isEqual(that.timestamp) - && Objects.equals(this.namespace, that.namespace); - } - - @Override - public int hashCode() { - return Objects.hash(domain, timestamp, namespace); + public static TimerData of( + String timerId, StateNamespace namespace, Instant timestamp, TimeDomain domain) { + return new AutoValue_TimerInternals_TimerData(timerId, namespace, timestamp, domain); } - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("namespace", namespace) - .add("timestamp", timestamp) - .add("domain", domain) - .toString(); + /** + * Construct a {@link TimerData} for the given parameters, where the timer ID is + * deterministically generated from the {@code timestamp} and {@code domain}. + */ + public static TimerData of(StateNamespace namespace, Instant timestamp, TimeDomain domain) { + String timerId = + new StringBuilder() + .append(domain.ordinal()) + .append(':') + .append(timestamp.getMillis()) + .toString(); + return of(timerId, namespace, timestamp, domain); } /** @@ -236,11 +203,11 @@ public int compareTo(TimerData that) { } ComparisonChain chain = ComparisonChain.start() - .compare(this.timestamp, that.getTimestamp()) - .compare(this.domain, that.domain); - if (chain.result() == 0 && !this.namespace.equals(that.namespace)) { + .compare(this.getTimestamp(), that.getTimestamp()) + .compare(this.getDomain(), that.getDomain()); + if (chain.result() == 0 && !this.getNamespace().equals(that.getNamespace())) { // Obtaining the stringKey may be expensive; only do so if required - chain = chain.compare(namespace.stringKey(), that.namespace.stringKey()); + chain = chain.compare(getNamespace().stringKey(), that.getNamespace().stringKey()); } return chain.result(); } @@ -275,20 +242,22 @@ private TimerDataCoder(Coder windowCoder) { public void encode(TimerData timer, OutputStream outStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); - STRING_CODER.encode(timer.namespace.stringKey(), outStream, nestedContext); - INSTANT_CODER.encode(timer.timestamp, outStream, nestedContext); - STRING_CODER.encode(timer.domain.name(), outStream, nestedContext); + STRING_CODER.encode(timer.getTimerId(), outStream, nestedContext); + STRING_CODER.encode(timer.getNamespace().stringKey(), outStream, nestedContext); + INSTANT_CODER.encode(timer.getTimestamp(), outStream, nestedContext); + STRING_CODER.encode(timer.getDomain().name(), outStream, nestedContext); } @Override public TimerData decode(InputStream inStream, Context context) throws CoderException, IOException { Context nestedContext = context.nested(); + String timerId = STRING_CODER.decode(inStream, nestedContext); StateNamespace namespace = StateNamespaces.fromString(STRING_CODER.decode(inStream, nestedContext), windowCoder); Instant timestamp = INSTANT_CODER.decode(inStream, nestedContext); TimeDomain domain = TimeDomain.valueOf(STRING_CODER.decode(inStream, nestedContext)); - return TimerData.of(namespace, timestamp, domain); + return TimerData.of(timerId, namespace, timestamp, domain); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index a3bb45a4cc14..60a90f5a07c6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -104,10 +104,10 @@ public void setTimer(StateNamespace namespace, String timerId, Instant target, } @Override - public void setTimer(TimerData timer) { - WindowTracing.trace("TestTimerInternals.setTimer: {}", timer); - if (existingTimers.add(timer)) { - queue(timer.getDomain()).add(timer); + public void setTimer(TimerData timerData) { + WindowTracing.trace("TestTimerInternals.setTimer: {}", timerData); + if (existingTimers.add(timerData)) { + queue(timerData.getDomain()).add(timerData); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java index e8ffdb34411e..7b56f1c75f09 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java @@ -45,12 +45,14 @@ public class TimerInternalsTest { public void testTimerDataCoder() throws Exception { CoderProperties.coderDecodeEncodeEqual( TimerDataCoder.of(GlobalWindow.Coder.INSTANCE), - TimerData.of(StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME)); + TimerData.of( + "arbitrary-id", StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME)); Coder windowCoder = IntervalWindow.getCoder(); CoderProperties.coderDecodeEncodeEqual( TimerDataCoder.of(windowCoder), TimerData.of( + "another-id", StateNamespaces.window( windowCoder, new IntervalWindow(new Instant(0), new Instant(100))), new Instant(99), TimeDomain.PROCESSING_TIME)); From b4ee8b730bffb31ee1178303f1dbd5058eb22a11 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 2 Dec 2016 10:56:15 -0800 Subject: [PATCH 062/279] Explicitly Throw in TransformExecutorTest --- .../runners/direct/TransformExecutorTest.java | 184 +++++++++--------- 1 file changed, 97 insertions(+), 87 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java index 85eff6598e34..08b1e188484a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java @@ -37,13 +37,10 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -63,7 +60,9 @@ public class TransformExecutorTest { @Rule public ExpectedException thrown = ExpectedException.none(); private PCollection created; - private PCollection> downstream; + + private AppliedPTransform createdProducer; + private AppliedPTransform downstreamProducer; private CountDownLatch evaluatorCompleted; @@ -88,15 +87,17 @@ public void setup() { TestPipeline p = TestPipeline.create(); created = p.apply(Create.of("foo", "spam", "third")); - downstream = created.apply(WithKeys.of(3)); + PCollection> downstream = created.apply(WithKeys.of(3)); + + createdProducer = created.getProducingTransformInternal(); + downstreamProducer = downstream.getProducingTransformInternal(); when(evaluationContext.getMetrics()).thenReturn(metrics); } @Test public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception { - final TransformResult result = - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); + final TransformResult result = StepTransformResult.withoutHold(createdProducer).build(); final AtomicBoolean finishCalled = new AtomicBoolean(false); TransformEvaluator evaluator = new TransformEvaluator() { @@ -112,8 +113,7 @@ public TransformResult finishBundle() throws Exception { } }; - when(registry.forApplication(created.getProducingTransformInternal(), null)) - .thenReturn(evaluator); + when(registry.forApplication(createdProducer, null)).thenReturn(evaluator); TransformExecutor executor = TransformExecutor.create( @@ -121,7 +121,7 @@ public TransformResult finishBundle() throws Exception { registry, Collections.emptyList(), null, - created.getProducingTransformInternal(), + createdProducer, completionCallback, transformEvaluationState); executor.run(); @@ -133,7 +133,7 @@ public TransformResult finishBundle() throws Exception { @Test public void nullTransformEvaluatorTerminates() throws Exception { - when(registry.forApplication(created.getProducingTransformInternal(), null)).thenReturn(null); + when(registry.forApplication(createdProducer, null)).thenReturn(null); TransformExecutor executor = TransformExecutor.create( @@ -141,7 +141,7 @@ public void nullTransformEvaluatorTerminates() throws Exception { registry, Collections.emptyList(), null, - created.getProducingTransformInternal(), + createdProducer, completionCallback, transformEvaluationState); executor.run(); @@ -154,7 +154,7 @@ public void nullTransformEvaluatorTerminates() throws Exception { @Test public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception { final TransformResult result = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(downstreamProducer).build(); final Collection> elementsProcessed = new ArrayList<>(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -175,8 +175,7 @@ public TransformResult finishBundle() throws Exception { WindowedValue third = WindowedValue.valueInGlobalWindow("third"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(foo).add(spam).add(third).commit(Instant.now()); - when(registry.forApplication(downstream.getProducingTransformInternal(), inputBundle)) - .thenReturn(evaluator); + when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); TransformExecutor executor = TransformExecutor.create( @@ -184,7 +183,7 @@ public TransformResult finishBundle() throws Exception { registry, Collections.emptyList(), inputBundle, - downstream.getProducingTransformInternal(), + downstreamProducer, completionCallback, transformEvaluationState); @@ -200,7 +199,7 @@ public TransformResult finishBundle() throws Exception { @Test public void processElementThrowsExceptionCallsback() throws Exception { final TransformResult result = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(downstreamProducer).build(); final Exception exception = new Exception(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -218,8 +217,7 @@ public TransformResult finishBundle() throws Exception { WindowedValue foo = WindowedValue.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(foo).commit(Instant.now()); - when(registry.forApplication(downstream.getProducingTransformInternal(), inputBundle)) - .thenReturn(evaluator); + when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); TransformExecutor executor = TransformExecutor.create( @@ -227,7 +225,7 @@ public TransformResult finishBundle() throws Exception { registry, Collections.emptyList(), inputBundle, - downstream.getProducingTransformInternal(), + downstreamProducer, completionCallback, transformEvaluationState); Executors.newSingleThreadExecutor().submit(executor); @@ -252,10 +250,8 @@ public TransformResult finishBundle() throws Exception { } }; - CommittedBundle inputBundle = - bundleFactory.createBundle(created).commit(Instant.now()); - when(registry.forApplication(downstream.getProducingTransformInternal(), inputBundle)) - .thenReturn(evaluator); + CommittedBundle inputBundle = bundleFactory.createBundle(created).commit(Instant.now()); + when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); TransformExecutor executor = TransformExecutor.create( @@ -263,7 +259,7 @@ public TransformResult finishBundle() throws Exception { registry, Collections.emptyList(), inputBundle, - downstream.getProducingTransformInternal(), + downstreamProducer, completionCallback, transformEvaluationState); Executors.newSingleThreadExecutor().submit(executor); @@ -277,7 +273,7 @@ public TransformResult finishBundle() throws Exception { @Test public void callWithEnforcementAppliesEnforcement() throws Exception { final TransformResult result = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(downstreamProducer).build(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -294,8 +290,7 @@ public TransformResult finishBundle() throws Exception { WindowedValue barElem = WindowedValue.valueInGlobalWindow("bar"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooElem).add(barElem).commit(Instant.now()); - when(registry.forApplication(downstream.getProducingTransformInternal(), inputBundle)) - .thenReturn(evaluator); + when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); TestEnforcementFactory enforcement = new TestEnforcementFactory(); TransformExecutor executor = @@ -304,7 +299,7 @@ public TransformResult finishBundle() throws Exception { registry, Collections.singleton(enforcement), inputBundle, - downstream.getProducingTransformInternal(), + downstreamProducer, completionCallback, transformEvaluationState); @@ -321,21 +316,8 @@ public TransformResult finishBundle() throws Exception { @Test public void callWithEnforcementThrowsOnFinishPropagates() throws Exception { - PCollection pcBytes = - created.apply( - new PTransform, PCollection>() { - @Override - public PCollection apply(PCollection input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(ByteArrayCoder.of()); - } - }); - final TransformResult result = - StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build(); - final CountDownLatch testLatch = new CountDownLatch(1); - final CountDownLatch evaluatorLatch = new CountDownLatch(1); + StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -344,62 +326,42 @@ public void processElement(WindowedValue element) throws Exception {} @Override public TransformResult finishBundle() throws Exception { - testLatch.countDown(); - evaluatorLatch.await(); return result; } }; - WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes()); - CommittedBundle inputBundle = - bundleFactory.createBundle(pcBytes).add(fooBytes).commit(Instant.now()); - when(registry.forApplication(pcBytes.getProducingTransformInternal(), inputBundle)) - .thenReturn(evaluator); + WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo"); + CommittedBundle inputBundle = + bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now()); + when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); - TransformExecutor executor = + TransformExecutor executor = TransformExecutor.create( evaluationContext, registry, - Collections.singleton(ImmutabilityEnforcementFactory.create()), + Collections.singleton( + new ThrowingEnforcementFactory(ThrowingEnforcementFactory.When.AFTER_BUNDLE)), inputBundle, - pcBytes.getProducingTransformInternal(), + downstreamProducer, completionCallback, transformEvaluationState); Future task = Executors.newSingleThreadExecutor().submit(executor); - testLatch.await(); - fooBytes.getValue()[0] = 'b'; - evaluatorLatch.countDown(); - thrown.expectCause(isA(IllegalMutationException.class)); + thrown.expectCause(isA(RuntimeException.class)); + thrown.expectMessage("afterFinish"); task.get(); } @Test public void callWithEnforcementThrowsOnElementPropagates() throws Exception { - PCollection pcBytes = - created.apply( - new PTransform, PCollection>() { - @Override - public PCollection apply(PCollection input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(ByteArrayCoder.of()); - } - }); - final TransformResult result = - StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build(); - final CountDownLatch testLatch = new CountDownLatch(1); - final CountDownLatch evaluatorLatch = new CountDownLatch(1); + StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); TransformEvaluator evaluator = new TransformEvaluator() { @Override - public void processElement(WindowedValue element) throws Exception { - testLatch.countDown(); - evaluatorLatch.await(); - } + public void processElement(WindowedValue element) throws Exception {} @Override public TransformResult finishBundle() throws Exception { @@ -407,28 +369,26 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes()); - CommittedBundle inputBundle = - bundleFactory.createBundle(pcBytes).add(fooBytes).commit(Instant.now()); - when(registry.forApplication(pcBytes.getProducingTransformInternal(), inputBundle)) - .thenReturn(evaluator); + WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo"); + CommittedBundle inputBundle = + bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now()); + when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); - TransformExecutor executor = + TransformExecutor executor = TransformExecutor.create( evaluationContext, registry, - Collections.singleton(ImmutabilityEnforcementFactory.create()), + Collections.singleton( + new ThrowingEnforcementFactory(ThrowingEnforcementFactory.When.AFTER_ELEMENT)), inputBundle, - pcBytes.getProducingTransformInternal(), + downstreamProducer, completionCallback, transformEvaluationState); Future task = Executors.newSingleThreadExecutor().submit(executor); - testLatch.await(); - fooBytes.getValue()[0] = 'b'; - evaluatorLatch.countDown(); - thrown.expectCause(isA(IllegalMutationException.class)); + thrown.expectCause(isA(RuntimeException.class)); + thrown.expectMessage("afterElement"); task.get(); } @@ -509,4 +469,54 @@ public void afterFinish( finishedBundles.add(result); } } + + private static class ThrowingEnforcementFactory implements ModelEnforcementFactory { + private final When when; + + private ThrowingEnforcementFactory(When when) { + this.when = when; + } + + enum When { + BEFORE_BUNDLE, + BEFORE_ELEMENT, + AFTER_ELEMENT, + AFTER_BUNDLE + } + + @Override + public ModelEnforcement forBundle( + CommittedBundle input, AppliedPTransform consumer) { + if (when == When.BEFORE_BUNDLE) { + throw new RuntimeException("forBundle"); + } + return new ThrowingEnforcement<>(); + } + + private class ThrowingEnforcement implements ModelEnforcement { + @Override + public void beforeElement(WindowedValue element) { + if (when == When.BEFORE_ELEMENT) { + throw new RuntimeException("beforeElement"); + } + } + + @Override + public void afterElement(WindowedValue element) { + if (when == When.AFTER_ELEMENT) { + throw new RuntimeException("afterElement"); + } + } + + @Override + public void afterFinish( + CommittedBundle input, + TransformResult result, + Iterable> outputs) { + if (when == When.AFTER_BUNDLE) { + throw new RuntimeException("afterFinish"); + } + } + } + } } From 8162cd29d97ef307b6fac588f453e4e39d70fca7 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 1 Dec 2016 15:39:30 -0800 Subject: [PATCH 063/279] Stop using Maps of Transforms in the DirectRunner Instead, add a "DirectGraph" class, which adds a layer of indirection to all lookup methods. Remove all remaining uses of getProducingTransformInternal, and instead use DirectGraph methods to obtain the producing transform. --- .../ConsumerTrackingPipelineVisitor.java | 108 +++++++----------- .../beam/runners/direct/DirectGraph.java | 89 +++++++++++++++ .../beam/runners/direct/DirectRunner.java | 31 +++-- .../runners/direct/EvaluationContext.java | 76 ++++-------- .../ExecutorServiceParallelExecutor.java | 15 +-- .../ImmutabilityCheckingBundleFactory.java | 21 ++-- .../beam/runners/direct/WatermarkManager.java | 50 ++++---- .../ConsumerTrackingPipelineVisitorTest.java | 98 ++++------------ .../runners/direct/EvaluationContextTest.java | 25 ++-- ...ImmutabilityCheckingBundleFactoryTest.java | 6 +- .../runners/direct/WatermarkManagerTest.java | 23 ++-- 11 files changed, 252 insertions(+), 290 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java index acfad16bf251..b9e77c5fc352 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java @@ -19,8 +19,8 @@ import static com.google.common.base.Preconditions.checkState; -import java.util.ArrayList; -import java.util.Collection; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.PTransform; 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; /** @@ -41,9 +42,13 @@ * input after the upstream transform has produced and committed output. */ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults { - private Map>> valueToConsumers = new HashMap<>(); - private Collection> rootTransforms = new ArrayList<>(); - private Collection> views = new ArrayList<>(); + private Map> producers = new HashMap<>(); + + private ListMultimap> primitiveConsumers = + ArrayListMultimap.create(); + + private Set> views = new HashSet<>(); + private Set> rootTransforms = new HashSet<>(); private Map, String> stepNames = new HashMap<>(); private Set toFinalize = new HashSet<>(); private int numTransforms = 0; @@ -81,81 +86,38 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { rootTransforms.add(appliedTransform); } else { for (PValue value : node.getInput().expand()) { - valueToConsumers.get(value).add(appliedTransform); + primitiveConsumers.put(value, appliedTransform); } } } - private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { - @SuppressWarnings({"rawtypes", "unchecked"}) - AppliedPTransform application = AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform()); - return application; - } - - @Override + @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { toFinalize.add(value); + + AppliedPTransform appliedTransform = getAppliedTransform(producer); + if (!producers.containsKey(value)) { + producers.put(value, appliedTransform); + } for (PValue expandedValue : value.expand()) { - valueToConsumers.put(expandedValue, new ArrayList>()); if (expandedValue instanceof PCollectionView) { views.add((PCollectionView) expandedValue); } - expandedValue.recordAsOutput(getAppliedTransform(producer)); + if (!producers.containsKey(expandedValue)) { + producers.put(value, appliedTransform); + } } - value.recordAsOutput(getAppliedTransform(producer)); - } - - private String genStepName() { - return String.format("s%s", numTransforms++); - } - - - /** - * Returns a mapping of each fully-expanded {@link PValue} to each - * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection - * returned from {@code getValueToCustomers().get(PValue)}, - * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}. - */ - public Map>> getValueToConsumers() { - checkState( - finalized, - "Can't call getValueToConsumers before the Pipeline has been completely traversed"); - - return valueToConsumers; } - /** - * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step - * name. - */ - public Map, String> getStepNames() { - checkState( - finalized, "Can't call getStepNames before the Pipeline has been completely traversed"); - - return stepNames; - } - - /** - * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes - * a {@link PInput} where the {@link PInput#expand()} returns an empty collection. - */ - public Collection> getRootTransforms() { - checkState( - finalized, - "Can't call getRootTransforms before the Pipeline has been completely traversed"); - - return rootTransforms; + private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { + @SuppressWarnings({"rawtypes", "unchecked"}) + AppliedPTransform application = AppliedPTransform.of( + node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform()); + return application; } - /** - * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited - * {@link Pipeline}. - */ - public Collection> getViews() { - checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed"); - - return views; + private String genStepName() { + return String.format("s%s", numTransforms++); } /** @@ -163,11 +125,21 @@ public Collection> getViews() { * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the * {@link Pipeline} is executed. */ - public Set getUnfinalizedPValues() { + public void finishSpecifyingRemainder() { checkState( finalized, - "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed"); + "Can't call finishSpecifyingRemainder before the Pipeline has been completely traversed"); + for (PValue unfinalized : toFinalize) { + unfinalized.finishSpecifying(); + } + } - return toFinalize; + /** + * Get the graph constructed by this {@link ConsumerTrackingPipelineVisitor}, which provides + * lookups for producers and consumers of {@link PValue PValues}. + */ + public DirectGraph getGraph() { + checkState(finalized, "Can't get a graph before the Pipeline has been completely traversed"); + return DirectGraph.create(producers, primitiveConsumers, views, rootTransforms, stepNames); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java new file mode 100644 index 000000000000..f208f6e2b55e --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java @@ -0,0 +1,89 @@ +/* + * 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.direct; + +import com.google.common.collect.ListMultimap; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +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; + +/** + * Methods for interacting with the underlying structure of a {@link Pipeline} that is being + * executed with the {@link DirectRunner}. + */ +class DirectGraph { + private final Map> producers; + private final ListMultimap> primitiveConsumers; + private final Set> views; + + private final Set> rootTransforms; + private final Map, String> stepNames; + + public static DirectGraph create( + Map> producers, + ListMultimap> primitiveConsumers, + Set> views, + Set> rootTransforms, + Map, String> stepNames) { + return new DirectGraph(producers, primitiveConsumers, views, rootTransforms, stepNames); + } + + private DirectGraph( + Map> producers, + ListMultimap> primitiveConsumers, + Set> views, + Set> rootTransforms, + Map, String> stepNames) { + this.producers = producers; + this.primitiveConsumers = primitiveConsumers; + this.views = views; + this.rootTransforms = rootTransforms; + this.stepNames = stepNames; + } + + public AppliedPTransform getProducer(PValue produced) { + return producers.get(produced); + } + + public List> getPrimitiveConsumers(PValue consumed) { + return primitiveConsumers.get(consumed); + } + + public Set> getRootTransforms() { + return rootTransforms; + } + + public Set> getViews() { + return views; + } + + public String getStepName(AppliedPTransform step) { + return stepNames.get(step); + } + + public Collection> getPrimitiveTransforms() { + return stepNames.keySet(); + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 82de9abe8bf0..0ad5836e3940 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -62,7 +62,6 @@ 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.joda.time.Duration; import org.joda.time.Instant; @@ -198,18 +197,18 @@ interface PCollectionViewWriter { enum Enforcement { ENCODABILITY { @Override - public boolean appliesTo(PTransform transform) { + public boolean appliesTo(PCollection collection, DirectGraph graph) { return true; } }, IMMUTABILITY { @Override - public boolean appliesTo(PTransform transform) { - return CONTAINS_UDF.contains(transform.getClass()); + public boolean appliesTo(PCollection collection, DirectGraph graph) { + return CONTAINS_UDF.contains(graph.getProducer(collection).getTransform().getClass()); } }; - public abstract boolean appliesTo(PTransform transform); + public abstract boolean appliesTo(PCollection collection, DirectGraph graph); //////////////////////////////////////////////////////////////////////////////////////////////// // Utilities for creating enforcements @@ -224,13 +223,13 @@ public static Set enabled(DirectOptions options) { return Collections.unmodifiableSet(enabled); } - public static BundleFactory bundleFactoryFor(Set enforcements) { + public static BundleFactory bundleFactoryFor(Set enforcements, DirectGraph graph) { BundleFactory bundleFactory = enforcements.contains(Enforcement.ENCODABILITY) ? CloningBundleFactory.create() : ImmutableListBundleFactory.create(); if (enforcements.contains(Enforcement.IMMUTABILITY)) { - bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory); + bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory, graph); } return bundleFactory; } @@ -301,9 +300,8 @@ public DirectPipelineResult run(Pipeline pipeline) { MetricsEnvironment.setMetricsSupported(true); ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor(); pipeline.traverseTopologically(consumerTrackingVisitor); - for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) { - unfinalized.finishSpecifying(); - } + consumerTrackingVisitor.finishSpecifyingRemainder(); + @SuppressWarnings("rawtypes") KeyedPValueTrackingVisitor keyedPValueVisitor = KeyedPValueTrackingVisitor.create( @@ -315,28 +313,25 @@ public DirectPipelineResult run(Pipeline pipeline) { DisplayDataValidator.validatePipeline(pipeline); + DirectGraph graph = consumerTrackingVisitor.getGraph(); EvaluationContext context = EvaluationContext.create( getPipelineOptions(), clockSupplier.get(), - Enforcement.bundleFactoryFor(enabledEnforcements), - consumerTrackingVisitor.getRootTransforms(), - consumerTrackingVisitor.getValueToConsumers(), - consumerTrackingVisitor.getStepNames(), - consumerTrackingVisitor.getViews()); + Enforcement.bundleFactoryFor(enabledEnforcements, graph), + graph); RootProviderRegistry rootInputProvider = RootProviderRegistry.defaultRegistry(context); TransformEvaluatorRegistry registry = TransformEvaluatorRegistry.defaultRegistry(context); PipelineExecutor executor = ExecutorServiceParallelExecutor.create( - options.getTargetParallelism(), - consumerTrackingVisitor.getValueToConsumers(), + options.getTargetParallelism(), graph, keyedPValueVisitor.getKeyedPValues(), rootInputProvider, registry, Enforcement.defaultModelEnforcements(enabledEnforcements), context); - executor.start(consumerTrackingVisitor.getRootTransforms()); + executor.start(graph.getRootTransforms()); Map, Collection>> aggregatorSteps = pipeline.getAggregatorSteps(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index 201aaedec526..b5a23d7cbc97 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -74,8 +74,10 @@ * can be executed. */ class EvaluationContext { - /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */ - private final Map, String> stepNames; + /** + * The graph representing this {@link Pipeline}. + */ + private final DirectGraph graph; /** The options that were used to create this {@link Pipeline}. */ private final DirectOptions options; @@ -99,36 +101,19 @@ class EvaluationContext { private final DirectMetrics metrics; public static EvaluationContext create( - DirectOptions options, - Clock clock, - BundleFactory bundleFactory, - Collection> rootTransforms, - Map>> valueToConsumers, - Map, String> stepNames, - Collection> views) { - return new EvaluationContext( - options, clock, bundleFactory, rootTransforms, valueToConsumers, stepNames, views); + DirectOptions options, Clock clock, BundleFactory bundleFactory, DirectGraph graph) { + return new EvaluationContext(options, clock, bundleFactory, graph); } private EvaluationContext( - DirectOptions options, - Clock clock, - BundleFactory bundleFactory, - Collection> rootTransforms, - Map>> valueToConsumers, - Map, String> stepNames, - Collection> views) { + DirectOptions options, Clock clock, BundleFactory bundleFactory, DirectGraph graph) { this.options = checkNotNull(options); this.clock = clock; this.bundleFactory = checkNotNull(bundleFactory); - checkNotNull(rootTransforms); - checkNotNull(valueToConsumers); - checkNotNull(stepNames); - checkNotNull(views); - this.stepNames = stepNames; + this.graph = checkNotNull(graph); - this.watermarkManager = WatermarkManager.create(clock, rootTransforms, valueToConsumers); - this.sideInputContainer = SideInputContainer.create(this, views); + this.watermarkManager = WatermarkManager.create(clock, graph); + this.sideInputContainer = SideInputContainer.create(this, graph.getViews()); this.applicationStateInternals = new ConcurrentHashMap<>(); this.mergedAggregators = AggregatorContainer.create(); @@ -211,7 +196,7 @@ private Iterable> commitBundles( ImmutableList.Builder> completed = ImmutableList.builder(); for (UncommittedBundle inProgress : bundles) { AppliedPTransform producing = - inProgress.getPCollection().getProducingTransformInternal(); + graph.getProducer(inProgress.getPCollection()); TransformWatermarks watermarks = watermarkManager.getWatermarks(producing); CommittedBundle committed = inProgress.commit(watermarks.getSynchronizedProcessingOutputTime()); @@ -225,7 +210,7 @@ private Iterable> commitBundles( } private void fireAllAvailableCallbacks() { - for (AppliedPTransform transform : stepNames.keySet()) { + for (AppliedPTransform transform : graph.getPrimitiveTransforms()) { fireAvailableCallbacks(transform); } } @@ -290,10 +275,10 @@ public void scheduleAfterOutputWouldBeProduced( BoundedWindow window, WindowingStrategy windowingStrategy, Runnable runnable) { - AppliedPTransform producing = getProducing(value); + AppliedPTransform producing = graph.getProducer(value); callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable); - fireAvailableCallbacks(lookupProducing(value)); + fireAvailableCallbacks(producing); } /** @@ -311,22 +296,6 @@ public void scheduleAfterWindowExpiration( fireAvailableCallbacks(producing); } - private AppliedPTransform getProducing(PValue value) { - if (value.getProducingTransformInternal() != null) { - return value.getProducingTransformInternal(); - } - return lookupProducing(value); - } - - private AppliedPTransform lookupProducing(PValue value) { - for (AppliedPTransform transform : stepNames.keySet()) { - if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) { - return transform; - } - } - return null; - } - /** * Get the options used by this {@link Pipeline}. */ @@ -347,18 +316,17 @@ public DirectExecutionContext getExecutionContext( watermarkManager.getWatermarks(application)); } - /** - * Get all of the steps used in this {@link Pipeline}. - */ - public Collection> getSteps() { - return stepNames.keySet(); - } /** * Get the Step Name for the provided application. */ - public String getStepName(AppliedPTransform application) { - return stepNames.get(application); + String getStepName(AppliedPTransform application) { + return graph.getStepName(application); + } + + /** Returns all of the steps in this {@link Pipeline}. */ + Collection> getSteps() { + return graph.getPrimitiveTransforms(); } /** @@ -450,7 +418,7 @@ public boolean isDone(AppliedPTransform transform) { * Returns true if all steps are done. */ public boolean isDone() { - for (AppliedPTransform transform : stepNames.keySet()) { + for (AppliedPTransform transform : graph.getPrimitiveTransforms()) { if (!isDone(transform)) { return false; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index b7908c5ed45d..929d09d8dee8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -69,7 +69,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor { private final int targetParallelism; private final ExecutorService executorService; - private final Map>> valueToConsumers; + private final DirectGraph graph; private final Set keyedPValues; private final RootProviderRegistry rootProviderRegistry; private final TransformEvaluatorRegistry registry; @@ -104,7 +104,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor { public static ExecutorServiceParallelExecutor create( int targetParallelism, - Map>> valueToConsumers, + DirectGraph graph, Set keyedPValues, RootProviderRegistry rootProviderRegistry, TransformEvaluatorRegistry registry, @@ -114,7 +114,7 @@ public static ExecutorServiceParallelExecutor create( EvaluationContext context) { return new ExecutorServiceParallelExecutor( targetParallelism, - valueToConsumers, + graph, keyedPValues, rootProviderRegistry, registry, @@ -124,7 +124,7 @@ public static ExecutorServiceParallelExecutor create( private ExecutorServiceParallelExecutor( int targetParallelism, - Map>> valueToConsumers, + DirectGraph graph, Set keyedPValues, RootProviderRegistry rootProviderRegistry, TransformEvaluatorRegistry registry, @@ -133,7 +133,7 @@ private ExecutorServiceParallelExecutor( EvaluationContext context) { this.targetParallelism = targetParallelism; this.executorService = Executors.newFixedThreadPool(targetParallelism); - this.valueToConsumers = valueToConsumers; + this.graph = graph; this.keyedPValues = keyedPValues; this.rootProviderRegistry = rootProviderRegistry; this.registry = registry; @@ -273,8 +273,9 @@ public final CommittedResult handleResult( CommittedBundle inputBundle, TransformResult result) { CommittedResult committedResult = evaluationContext.handleResult(inputBundle, timers, result); for (CommittedBundle outputBundle : committedResult.getOutputs()) { - allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle, - valueToConsumers.get(outputBundle.getPCollection()))); + allUpdates.offer( + ExecutorUpdate.fromBundle( + outputBundle, graph.getPrimitiveConsumers(outputBundle.getPCollection()))); } CommittedBundle unprocessedInputs = committedResult.getUnprocessedInputs(); if (unprocessedInputs != null && !Iterables.isEmpty(unprocessedInputs.getElements())) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java index 4f72f68fd092..8d77e25a7739 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java @@ -46,17 +46,20 @@ */ class ImmutabilityCheckingBundleFactory implements BundleFactory { /** - * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying - * {@link BundleFactory} to create the output bundle. + * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying {@link + * BundleFactory} to create the output bundle. */ - public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) { - return new ImmutabilityCheckingBundleFactory(underlying); + public static ImmutabilityCheckingBundleFactory create( + BundleFactory underlying, DirectGraph graph) { + return new ImmutabilityCheckingBundleFactory(underlying, graph); } private final BundleFactory underlying; + private final DirectGraph graph; - private ImmutabilityCheckingBundleFactory(BundleFactory underlying) { + private ImmutabilityCheckingBundleFactory(BundleFactory underlying, DirectGraph graph) { this.underlying = checkNotNull(underlying); + this.graph = graph; } /** @@ -72,7 +75,7 @@ public UncommittedBundle createRootBundle() { @Override public UncommittedBundle createBundle(PCollection output) { - if (Enforcement.IMMUTABILITY.appliesTo(output.getProducingTransformInternal().getTransform())) { + if (Enforcement.IMMUTABILITY.appliesTo(output, graph)) { return new ImmutabilityEnforcingBundle<>(underlying.createBundle(output)); } return underlying.createBundle(output); @@ -81,13 +84,13 @@ public UncommittedBundle createBundle(PCollection output) { @Override public UncommittedBundle createKeyedBundle( StructuralKey key, PCollection output) { - if (Enforcement.IMMUTABILITY.appliesTo(output.getProducingTransformInternal().getTransform())) { + if (Enforcement.IMMUTABILITY.appliesTo(output, graph)) { return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(key, output)); } return underlying.createKeyedBundle(key, output); } - private static class ImmutabilityEnforcingBundle implements UncommittedBundle { + private class ImmutabilityEnforcingBundle implements UncommittedBundle { private final UncommittedBundle underlying; private final SetMultimap, MutationDetector> mutationDetectors; private Coder coder; @@ -125,7 +128,7 @@ public CommittedBundle commit(Instant synchronizedProcessingTime) { String.format( "PTransform %s mutated value %s after it was output (new value was %s)." + " Values must not be mutated in any way after being output.", - underlying.getPCollection().getProducingTransformInternal().getFullName(), + graph.getProducer(underlying.getPCollection()).getFullName(), exn.getSavedValue(), exn.getNewValue()), exn.getSavedValue(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index a53c11c72176..247b1cce9ab4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -669,10 +669,10 @@ private static Map, List> extractFiredTimers( private final Clock clock; /** - * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications} - * that consume that {@link PCollection}. + * The {@link DirectGraph} representing the {@link Pipeline} this {@link WatermarkManager} tracks + * watermarks for. */ - private final Map>> consumers; + private final DirectGraph graph; /** * The input and output watermark of each {@link AppliedPTransform}. @@ -697,27 +697,21 @@ private static Map, List> extractFiredTimers( private final Set> pendingRefreshes; /** - * Creates a new {@link WatermarkManager}. All watermarks within the newly created - * {@link WatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the - * minimum watermark, with no watermark holds or pending elements. + * Creates a new {@link WatermarkManager}. All watermarks within the newly created {@link + * WatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the minimum watermark, + * with no watermark holds or pending elements. * - * @param rootTransforms the root-level transforms of the {@link Pipeline} - * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the - * transforms that consume it as a part of their input + * @param clock the clock to use to determine processing time + * @param graph the graph representing this pipeline */ - public static WatermarkManager create( - Clock clock, - Collection> rootTransforms, - Map>> consumers) { - return new WatermarkManager(clock, rootTransforms, consumers); + public static WatermarkManager create(Clock clock, DirectGraph graph) { + return new WatermarkManager(clock, graph); } - private WatermarkManager( - Clock clock, - Collection> rootTransforms, - Map>> consumers) { + private WatermarkManager(Clock clock, DirectGraph graph) { this.clock = clock; - this.consumers = consumers; + this.graph = graph; + this.pendingUpdates = new ConcurrentLinkedQueue<>(); this.refreshLock = new ReentrantLock(); @@ -725,13 +719,11 @@ private WatermarkManager( transformToWatermarks = new HashMap<>(); - for (AppliedPTransform rootTransform : rootTransforms) { + for (AppliedPTransform rootTransform : graph.getRootTransforms()) { getTransformWatermark(rootTransform); } - for (Collection> intermediateTransforms : consumers.values()) { - for (AppliedPTransform transform : intermediateTransforms) { - getTransformWatermark(transform); - } + for (AppliedPTransform primitiveTransform : graph.getPrimitiveTransforms()) { + getTransformWatermark(primitiveTransform); } } @@ -769,8 +761,7 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) } for (PValue pvalue : inputs) { Watermark producerOutputWatermark = - getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark; + getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; inputWatermarksBuilder.add(producerOutputWatermark); } List inputCollectionWatermarks = inputWatermarksBuilder.build(); @@ -920,7 +911,8 @@ private void updatePending( // do not share a Mutex within this call and thus can be interleaved with external calls to // refresh. for (CommittedBundle bundle : result.getOutputs()) { - for (AppliedPTransform consumer : consumers.get(bundle.getPCollection())) { + for (AppliedPTransform consumer : + graph.getPrimitiveConsumers(bundle.getPCollection())) { TransformWatermarks watermarks = transformToWatermarks.get(consumer); watermarks.addPending(bundle); } @@ -968,7 +960,7 @@ synchronized void refreshAll() { if (updateResult.isAdvanced()) { Set> additionalRefreshes = new HashSet<>(); for (PValue outputPValue : toRefresh.getOutput().expand()) { - additionalRefreshes.addAll(consumers.get(outputPValue)); + additionalRefreshes.addAll(graph.getPrimitiveConsumers(outputPValue)); } return additionalRefreshes; } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java index f7f4b710cf60..02fe007d7fea 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.direct; import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import java.io.Serializable; @@ -36,7 +38,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.PValue; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -72,7 +73,7 @@ public void processElement(DoFn.ProcessContext c) p.apply("singletonCreate", Create.of(1, 2, 3)).apply(View.asSingleton()); p.traverseTopologically(visitor); assertThat( - visitor.getViews(), + visitor.getGraph().getViews(), Matchers.>containsInAnyOrder(listView, singletonView)); } @@ -83,7 +84,7 @@ public void getRootTransformsContainsPBegins() { PCollection unCounted = p.apply(CountingInput.unbounded()); p.traverseTopologically(visitor); assertThat( - visitor.getRootTransforms(), + visitor.getGraph().getRootTransforms(), Matchers.>containsInAnyOrder( created.getProducingTransformInternal(), counted.getProducingTransformInternal(), @@ -96,7 +97,7 @@ public void getRootTransformsContainsEmptyFlatten() { PCollectionList.empty(p).apply(Flatten.pCollections()); p.traverseTopologically(visitor); assertThat( - visitor.getRootTransforms(), + visitor.getGraph().getRootTransforms(), Matchers.>containsInAnyOrder( empty.getProducingTransformInternal())); } @@ -121,15 +122,15 @@ public void processElement(DoFn.ProcessContext c) p.traverseTopologically(visitor); assertThat( - visitor.getValueToConsumers().get(created), + visitor.getGraph().getPrimitiveConsumers(created), Matchers.>containsInAnyOrder( transformed.getProducingTransformInternal(), flattened.getProducingTransformInternal())); assertThat( - visitor.getValueToConsumers().get(transformed), + visitor.getGraph().getPrimitiveConsumers(transformed), Matchers.>containsInAnyOrder( flattened.getProducingTransformInternal())); - assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable()); + assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable()); } @Test @@ -142,11 +143,11 @@ public void getValueToConsumersWithDuplicateInputSucceeds() { p.traverseTopologically(visitor); assertThat( - visitor.getValueToConsumers().get(created), + visitor.getGraph().getPrimitiveConsumers(created), Matchers.>containsInAnyOrder( flattened.getProducingTransformInternal(), flattened.getProducingTransformInternal())); - assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable()); + assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable()); } @Test @@ -163,32 +164,11 @@ public void processElement(DoFn.ProcessContext c) } })); - p.traverseTopologically(visitor); - assertThat(visitor.getUnfinalizedPValues(), Matchers.contains(transformed)); - } - - @Test - public void getUnfinalizedPValuesEmpty() { - p.apply(Create.of("1", "2", "3")) - .apply( - ParDo.of( - new DoFn() { - @ProcessElement - public void processElement(DoFn.ProcessContext c) - throws Exception { - c.output(Integer.toString(c.element().length())); - } - })) - .apply( - new PTransform() { - @Override - public PDone apply(PInput input) { - return PDone.in(input.getPipeline()); - } - }); + assertThat(transformed.isFinishedSpecifyingInternal(), is(false)); p.traverseTopologically(visitor); - assertThat(visitor.getUnfinalizedPValues(), emptyIterable()); + visitor.finishSpecifyingRemainder(); + assertThat(transformed.isFinishedSpecifyingInternal(), is(true)); } @Test @@ -214,18 +194,12 @@ public PDone apply(PInput input) { }); p.traverseTopologically(visitor); - assertThat( - visitor.getStepNames(), - Matchers., String>hasEntry( - created.getProducingTransformInternal(), "s0")); - assertThat( - visitor.getStepNames(), - Matchers., String>hasEntry( - transformed.getProducingTransformInternal(), "s1")); - assertThat( - visitor.getStepNames(), - Matchers., String>hasEntry( - finished.getProducingTransformInternal(), "s2")); + DirectGraph graph = visitor.getGraph(); + assertThat(graph.getStepName(graph.getProducer(created)), equalTo("s0")); + assertThat(graph.getStepName(graph.getProducer(transformed)), equalTo("s1")); + // finished doesn't have a producer, because it's not a PValue. + // TODO: Demonstrate that PCollectionList/Tuple and other composite PValues are either safe to + // use, or make them so. } @Test @@ -248,40 +222,18 @@ public void traverseIndependentPathsSucceeds() { } @Test - public void getRootTransformsWithoutVisitingThrows() { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("completely traversed"); - thrown.expectMessage("getRootTransforms"); - visitor.getRootTransforms(); - } - @Test - public void getStepNamesWithoutVisitingThrows() { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("completely traversed"); - thrown.expectMessage("getStepNames"); - visitor.getStepNames(); - } - @Test - public void getUnfinalizedPValuesWithoutVisitingThrows() { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("completely traversed"); - thrown.expectMessage("getUnfinalizedPValues"); - visitor.getUnfinalizedPValues(); - } - - @Test - public void getValueToConsumersWithoutVisitingThrows() { + public void getGraphWithoutVisitingThrows() { thrown.expect(IllegalStateException.class); thrown.expectMessage("completely traversed"); - thrown.expectMessage("getValueToConsumers"); - visitor.getValueToConsumers(); + thrown.expectMessage("get a graph"); + visitor.getGraph(); } @Test - public void getViewsWithoutVisitingThrows() { + public void finishSpecifyingRemainderWithoutVisitingThrows() { thrown.expect(IllegalStateException.class); thrown.expectMessage("completely traversed"); - thrown.expectMessage("getViews"); - visitor.getViews(); + thrown.expectMessage("finishSpecifyingRemainder"); + visitor.finishSpecifyingRemainder(); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 9a3959dab2c2..1c2bf1452a99 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -29,7 +29,6 @@ import com.google.common.collect.Iterables; import java.util.Collection; import java.util.Collections; -import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; @@ -67,7 +66,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValue; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -87,10 +85,9 @@ public class EvaluationContextTest { private PCollection> downstream; private PCollectionView> view; private PCollection unbounded; - private Collection> rootTransforms; - private Map>> valueToConsumers; private BundleFactory bundleFactory; + private DirectGraph graph; @Before public void setup() { @@ -106,20 +103,12 @@ public void setup() { ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor(); p.traverseTopologically(cVis); - rootTransforms = cVis.getRootTransforms(); - valueToConsumers = cVis.getValueToConsumers(); bundleFactory = ImmutableListBundleFactory.create(); - + graph = cVis.getGraph(); context = EvaluationContext.create( - runner.getPipelineOptions(), - NanosOffsetClock.create(), - ImmutableListBundleFactory.create(), - rootTransforms, - valueToConsumers, - cVis.getStepNames(), - cVis.getViews()); + runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph); } @Test @@ -427,13 +416,13 @@ public void isDoneWithUnboundedPCollectionAndShutdown() { @Test public void isDoneWithUnboundedPCollectionAndNotShutdown() { context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false); - assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false)); + assertThat(context.isDone(graph.getProducer(unbounded)), is(false)); context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build()); - assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false)); + StepTransformResult.withoutHold(graph.getProducer(unbounded)).build()); + assertThat(context.isDone(graph.getProducer(unbounded)), is(false)); } @Test @@ -472,7 +461,7 @@ public void isDoneWithPartiallyDone() { StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build()); assertThat(context.isDone(), is(false)); - for (AppliedPTransform consumers : valueToConsumers.get(created)) { + for (AppliedPTransform consumers : graph.getPrimitiveConsumers(created)) { context.handleResult( committedBundle, ImmutableList.of(), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index ea441256d560..e7e1e625558d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -56,7 +56,11 @@ public void setup() { TestPipeline p = TestPipeline.create(); created = p.apply(Create.of().withCoder(ByteArrayCoder.of())); transformed = created.apply(ParDo.of(new IdentityDoFn())); - factory = ImmutabilityCheckingBundleFactory.create(ImmutableListBundleFactory.create()); + ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor(); + p.traverseTopologically(visitor); + factory = + ImmutabilityCheckingBundleFactory.create( + ImmutableListBundleFactory.create(), visitor.getGraph()); } @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 2e8ab84224ef..5cde4d62cff7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -94,6 +94,7 @@ public class WatermarkManagerTest implements Serializable { private transient WatermarkManager manager; private transient BundleFactory bundleFactory; + private DirectGraph graph; @Before public void setup() { @@ -139,8 +140,11 @@ public void processElement(ProcessContext c) throws Exception { consumers.put(flattened, Collections.>emptyList()); clock = MockClock.fromInstant(new Instant(1000)); + ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor(); + p.traverseTopologically(visitor); + graph = visitor.getGraph(); - manager = WatermarkManager.create(clock, rootTransforms, consumers); + manager = WatermarkManager.create(clock, graph); bundleFactory = ImmutableListBundleFactory.create(); } @@ -305,20 +309,13 @@ public void getWatermarkMultiIdenticalInput() { PCollection created = p.apply(Create.of(1, 2, 3)); PCollection multiConsumer = PCollectionList.of(created).and(created).apply(Flatten.pCollections()); - AppliedPTransform theFlatten = multiConsumer.getProducingTransformInternal(); + ConsumerTrackingPipelineVisitor trackingVisitor = new ConsumerTrackingPipelineVisitor(); + p.traverseTopologically(trackingVisitor); + DirectGraph graph = trackingVisitor.getGraph(); - Map>> valueToConsumers = - ImmutableMap.>>builder() - .put(created, ImmutableList.>of(theFlatten, theFlatten)) - .put(multiConsumer, Collections.>emptyList()) - .build(); + AppliedPTransform theFlatten = graph.getProducer(multiConsumer); - WatermarkManager tstMgr = - WatermarkManager.create( - clock, - Collections.>singleton( - created.getProducingTransformInternal()), - valueToConsumers); + WatermarkManager tstMgr = WatermarkManager.create(clock, graph); CommittedBundle root = bundleFactory .createRootBundle() From 662416a4e176cca252c0d6fde1bf4252aeaa56c0 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 2 Dec 2016 10:07:05 -0800 Subject: [PATCH 064/279] Rename ConsumerTrackingPipelineVisitor to DirectGraphVisitor Reduce visibility of Visitor. --- ...ackingPipelineVisitor.java => DirectGraphVisitor.java} | 8 ++++---- .../java/org/apache/beam/runners/direct/DirectRunner.java | 8 ++++---- ...pelineVisitorTest.java => DirectGraphVisitorTest.java} | 8 ++++---- .../apache/beam/runners/direct/EvaluationContextTest.java | 6 +++--- .../direct/ImmutabilityCheckingBundleFactoryTest.java | 2 +- .../apache/beam/runners/direct/WatermarkManagerTest.java | 8 ++++---- 6 files changed, 20 insertions(+), 20 deletions(-) rename runners/direct-java/src/main/java/org/apache/beam/runners/direct/{ConsumerTrackingPipelineVisitor.java => DirectGraphVisitor.java} (94%) rename runners/direct-java/src/test/java/org/apache/beam/runners/direct/{ConsumerTrackingPipelineVisitorTest.java => DirectGraphVisitorTest.java} (96%) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java similarity index 94% rename from runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java rename to runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index b9e77c5fc352..cd9d1201dac6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -41,7 +41,7 @@ * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume * input after the upstream transform has produced and committed output. */ -public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults { +class DirectGraphVisitor extends PipelineVisitor.Defaults { private Map> producers = new HashMap<>(); private ListMultimap> primitiveConsumers = @@ -61,7 +61,7 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { "Attempting to traverse a pipeline (node %s) with a %s " + "which has already visited a Pipeline and is finalized", node.getFullName(), - ConsumerTrackingPipelineVisitor.class.getSimpleName()); + getClass().getSimpleName()); return CompositeBehavior.ENTER_TRANSFORM; } @@ -71,7 +71,7 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { !finalized, "Attempting to traverse a pipeline (node %s) with a %s which is already finalized", node.getFullName(), - ConsumerTrackingPipelineVisitor.class.getSimpleName()); + getClass().getSimpleName()); if (node.isRootNode()) { finalized = true; } @@ -135,7 +135,7 @@ public void finishSpecifyingRemainder() { } /** - * Get the graph constructed by this {@link ConsumerTrackingPipelineVisitor}, which provides + * Get the graph constructed by this {@link DirectGraphVisitor}, which provides * lookups for producers and consumers of {@link PValue PValues}. */ public DirectGraph getGraph() { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 0ad5836e3940..2f84356d6e91 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -298,9 +298,9 @@ public OutputT apply( @Override public DirectPipelineResult run(Pipeline pipeline) { MetricsEnvironment.setMetricsSupported(true); - ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor(); - pipeline.traverseTopologically(consumerTrackingVisitor); - consumerTrackingVisitor.finishSpecifyingRemainder(); + DirectGraphVisitor graphVisitor = new DirectGraphVisitor(); + pipeline.traverseTopologically(graphVisitor); + graphVisitor.finishSpecifyingRemainder(); @SuppressWarnings("rawtypes") KeyedPValueTrackingVisitor keyedPValueVisitor = @@ -313,7 +313,7 @@ public DirectPipelineResult run(Pipeline pipeline) { DisplayDataValidator.validatePipeline(pipeline); - DirectGraph graph = consumerTrackingVisitor.getGraph(); + DirectGraph graph = graphVisitor.getGraph(); EvaluationContext context = EvaluationContext.create( getPipelineOptions(), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java similarity index 96% rename from runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java rename to runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java index 02fe007d7fea..d218a81e2dc2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java @@ -46,14 +46,14 @@ import org.junit.runners.JUnit4; /** - * Tests for {@link ConsumerTrackingPipelineVisitor}. + * Tests for {@link DirectGraphVisitor}. */ @RunWith(JUnit4.class) -public class ConsumerTrackingPipelineVisitorTest implements Serializable { +public class DirectGraphVisitorTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); private transient TestPipeline p = TestPipeline.create(); - private transient ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor(); + private transient DirectGraphVisitor visitor = new DirectGraphVisitor(); @Test public void getViewsReturnsViews() { @@ -208,7 +208,7 @@ public void traverseMultipleTimesThrows() { p.traverseTopologically(visitor); thrown.expect(IllegalStateException.class); - thrown.expectMessage(ConsumerTrackingPipelineVisitor.class.getSimpleName()); + thrown.expectMessage(DirectGraphVisitor.class.getSimpleName()); thrown.expectMessage("is finalized"); p.traverseTopologically(visitor); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 1c2bf1452a99..17cdea1440a2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -101,11 +101,11 @@ public void setup() { view = created.apply(View.asIterable()); unbounded = p.apply(CountingInput.unbounded()); - ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor(); - p.traverseTopologically(cVis); + DirectGraphVisitor graphVisitor = new DirectGraphVisitor(); + p.traverseTopologically(graphVisitor); bundleFactory = ImmutableListBundleFactory.create(); - graph = cVis.getGraph(); + graph = graphVisitor.getGraph(); context = EvaluationContext.create( runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index e7e1e625558d..6ab8aea2871f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -56,7 +56,7 @@ public void setup() { TestPipeline p = TestPipeline.create(); created = p.apply(Create.of().withCoder(ByteArrayCoder.of())); transformed = created.apply(ParDo.of(new IdentityDoFn())); - ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor(); + DirectGraphVisitor visitor = new DirectGraphVisitor(); p.traverseTopologically(visitor); factory = ImmutabilityCheckingBundleFactory.create( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 5cde4d62cff7..076e0fb1643a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -140,7 +140,7 @@ public void processElement(ProcessContext c) throws Exception { consumers.put(flattened, Collections.>emptyList()); clock = MockClock.fromInstant(new Instant(1000)); - ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor(); + DirectGraphVisitor visitor = new DirectGraphVisitor(); p.traverseTopologically(visitor); graph = visitor.getGraph(); @@ -309,9 +309,9 @@ public void getWatermarkMultiIdenticalInput() { PCollection created = p.apply(Create.of(1, 2, 3)); PCollection multiConsumer = PCollectionList.of(created).and(created).apply(Flatten.pCollections()); - ConsumerTrackingPipelineVisitor trackingVisitor = new ConsumerTrackingPipelineVisitor(); - p.traverseTopologically(trackingVisitor); - DirectGraph graph = trackingVisitor.getGraph(); + DirectGraphVisitor graphVisitor = new DirectGraphVisitor(); + p.traverseTopologically(graphVisitor); + DirectGraph graph = graphVisitor.getGraph(); AppliedPTransform theFlatten = graph.getProducer(multiConsumer); From e3dca4cab6914166465c70f5f0b4be4f06ddd088 Mon Sep 17 00:00:00 2001 From: Neelesh Srinivas Salian Date: Thu, 1 Dec 2016 20:28:43 -0800 Subject: [PATCH 065/279] BEAM-879: Changing DeDupExample to DistinctExample --- .../cookbook/{DeDupExample.java => DistinctExample.java} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename examples/java/src/main/java/org/apache/beam/examples/cookbook/{DeDupExample.java => DistinctExample.java} (97%) diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java similarity index 97% rename from examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java rename to examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java index 34fb9016828c..9670b7f49265 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java @@ -48,10 +48,10 @@ *

    The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be * overridden with {@code --input}. */ -public class DeDupExample { +public class DistinctExample { /** - * Options supported by {@link DeDupExample}. + * Options supported by {@link DistinctExample}. * *

    Inherits standard configuration options. */ From 96455768568616141a95833380f37c478a989397 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 18 Nov 2016 13:10:22 -0800 Subject: [PATCH 066/279] Makes DoFnTester use new DoFn internally. There were 2 remaining users of DoFnTester.of(OldDoFn): - SplittableParDo.ProcessElements: this is fixed in https://github.com/apache/incubator-beam/pull/1261 - GroupAlsoByWindowsProperties: this one is harder. Various GABWDoFn's use OldDoFn.windowingInternals, and we can't pass that through a new DoFn. So instead I removed usage of DoFnTester from GroupAlsoByWindowsProperties in favor of a tiny hand-coded solution. So after #1261 DoFnTester.of(OldDoFn) can be deleted. --- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 6 +- .../core/GroupByKeyViaGroupByKeyOnly.java | 22 +- .../core/GroupAlsoByWindowsProperties.java | 590 ++++++++++-------- .../beam/sdk/transforms/DoFnAdapters.java | 2 + .../beam/sdk/transforms/DoFnTester.java | 130 ++-- .../sdk/transforms/reflect/DoFnInvokers.java | 11 - .../beam/sdk/transforms/DoFnTesterTest.java | 4 +- 7 files changed, 394 insertions(+), 371 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index f8f62073c6ae..b4b366cb61f4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.WindowedValue; @@ -30,7 +29,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.util.state.TimerCallback; -import org.apache.beam.sdk.values.KV; import org.joda.time.Instant; /** @@ -55,9 +53,7 @@ public GroupAlsoByWindowsViaOutputBufferDoFn( } @Override - public void processElement( - OldDoFn>>, KV>.ProcessContext c) - throws Exception { + public void processElement(ProcessContext c) throws Exception { K key = c.element().getKey(); // Used with Batch, we know that all the data is available for this key. We can't use the // timer manager from the context because it doesn't exist. So we create one and emulate the diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index 79d22524c387..43047ca0a81f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java @@ -26,15 +26,13 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -135,10 +133,9 @@ public PCollection>>> apply( return input .apply( ParDo.of( - new OldDoFn< - KV>>, - KV>>>() { - @Override + new DoFn>>, + KV>>>() { + @ProcessElement public void processElement(ProcessContext c) { KV>> kvs = c.element(); K key = kvs.getKey(); @@ -251,16 +248,5 @@ public PCollection>> apply( input.getPipeline(), windowingStrategy, input.isBounded()) .setCoder(outputKvCoder); } - - private - GroupAlsoByWindowsViaOutputBufferDoFn, W> groupAlsoByWindowsFn( - WindowingStrategy strategy, - StateInternalsFactory stateInternalsFactory, - Coder inputIterableElementValueCoder) { - return new GroupAlsoByWindowsViaOutputBufferDoFn, W>( - strategy, - stateInternalsFactory, - SystemReduceFn.buffering(inputIterableElementValueCoder)); - } } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java index d1e0c6814392..97b67c653c9c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java @@ -23,50 +23,60 @@ import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertThat; +import com.google.common.base.Predicate; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.DoFnTester; -import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +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.InMemoryStateInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; import org.joda.time.Instant; /** * Properties of {@link GroupAlsoByWindowsDoFn}. * - *

    Some properties may not hold of some implementations, due to restrictions on the context - * in which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not + *

    Some properties may not hold of some implementations, due to restrictions on the context in + * which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not * support merging windows. */ public class GroupAlsoByWindowsProperties { /** - * A factory of {@link GroupAlsoByWindowsDoFn} so that the various properties can provide - * the appropriate windowing strategy under test. + * A factory of {@link GroupAlsoByWindowsDoFn} so that the various properties can provide the + * appropriate windowing strategy under test. */ public interface GroupAlsoByWindowsDoFnFactory { - GroupAlsoByWindowsDoFn - forStrategy(WindowingStrategy strategy, StateInternalsFactory stateInternalsFactory); + GroupAlsoByWindowsDoFn forStrategy( + WindowingStrategy strategy, StateInternalsFactory stateInternalsFactory); } /** @@ -76,8 +86,7 @@ public interface GroupAlsoByWindowsDoFnFactory { *

    The input type is deliberately left as a wildcard, since it is not relevant. */ public static void emptyInputEmptyOutput( - GroupAlsoByWindowsDoFnFactory gabwFactory) - throws Exception { + GroupAlsoByWindowsDoFnFactory gabwFactory) throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))); @@ -87,13 +96,14 @@ public static void emptyInputEmptyOutput( @SuppressWarnings("unchecked") K fakeKey = (K) "this key should never be used"; - DoFnTester>>, KV> result = runGABW( - gabwFactory, - windowingStrategy, - fakeKey, - Collections.>emptyList()); + List>> result = + runGABW( + gabwFactory, + windowingStrategy, + fakeKey, + Collections.>emptyList()); - assertThat(result.peekOutputElements(), hasSize(0)); + assertThat(result, hasSize(0)); } /** @@ -102,38 +112,32 @@ public static void emptyInputEmptyOutput( */ public static void groupsElementsIntoFixedWindows( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "key", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "key", WindowedValue.of( - "v1", - new Instant(1), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v2", - new Instant(2), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(13), - Arrays.asList(window(10, 20)), - PaneInfo.NO_FIRING)); + "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10))); + getOnlyElementInWindow(result, window(0, 10)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp())); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20))); + getOnlyElementInWindow(result, window(10, 20)); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp())); } @@ -146,14 +150,17 @@ public static void groupsElementsIntoFixedWindows( */ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { - WindowingStrategy windowingStrategy = WindowingStrategy.of( - SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))) - .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()); + WindowingStrategy windowingStrategy = + WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))) + .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "key", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "key", WindowedValue.of( "v1", new Instant(5), @@ -165,21 +172,21 @@ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(3)); + assertThat(result, hasSize(3)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10))); + getOnlyElementInWindow(result, window(-10, 10)); assertThat(item0.getValue().getValue(), contains("v1")); assertThat(item0.getTimestamp(), equalTo(new Instant(5))); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20))); + getOnlyElementInWindow(result, window(0, 20)); assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2")); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item1.getTimestamp(), equalTo(new Instant(10))); TimestampedValue>> item2 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30))); + getOnlyElementInWindow(result, window(10, 30)); assertThat(item2.getValue().getValue(), contains("v2")); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item2.getTimestamp(), equalTo(new Instant(20))); @@ -194,14 +201,17 @@ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( public static void combinesElementsInSlidingWindows( GroupAlsoByWindowsDoFnFactory gabwFactory, CombineFn combineFn) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()); - DoFnTester>>, KV> result = - runGABW(gabwFactory, windowingStrategy, "k", + List>> result = + runGABW( + gabwFactory, + windowingStrategy, + "k", WindowedValue.of( 1L, new Instant(5), @@ -218,23 +228,20 @@ public static void combinesElementsInSlidingWindows( Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(3)); + assertThat(result, hasSize(3)); - TimestampedValue> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10))); + TimestampedValue> item0 = getOnlyElementInWindow(result, window(-10, 10)); assertThat(item0.getValue().getKey(), equalTo("k")); assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L)))); assertThat(item0.getTimestamp(), equalTo(new Instant(5L))); - TimestampedValue> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20))); + TimestampedValue> item1 = getOnlyElementInWindow(result, window(0, 20)); assertThat(item1.getValue().getKey(), equalTo("k")); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L)))); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item1.getTimestamp(), equalTo(new Instant(10L))); - TimestampedValue> item2 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30))); + TimestampedValue> item2 = getOnlyElementInWindow(result, window(10, 30)); assertThat(item2.getValue().getKey(), equalTo("k")); assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L)))); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window @@ -247,79 +254,63 @@ public static void combinesElementsInSlidingWindows( */ public static void groupsIntoOverlappingNonmergingWindows( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "key", - WindowedValue.of( - "v1", - new Instant(1), - Arrays.asList(window(0, 5)), - PaneInfo.NO_FIRING), - WindowedValue.of( - "v2", - new Instant(4), - Arrays.asList(window(1, 5)), - PaneInfo.NO_FIRING), + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "key", + WindowedValue.of("v1", new Instant(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), + WindowedValue.of("v2", new Instant(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(4), - Arrays.asList(window(0, 5)), - PaneInfo.NO_FIRING)); + "v3", new Instant(4), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 5))); + getOnlyElementInWindow(result, window(0, 5)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v3")); assertThat(item0.getTimestamp(), equalTo(window(1, 5).maxTimestamp())); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(1, 5))); + getOnlyElementInWindow(result, window(1, 5)); assertThat(item1.getValue().getValue(), contains("v2")); assertThat(item1.getTimestamp(), equalTo(window(0, 5).maxTimestamp())); } - /** - * Tests that the given GABW implementation correctly groups elements into merged sessions. - */ + /** Tests that the given GABW implementation correctly groups elements into merged sessions. */ public static void groupsElementsInMergedSessions( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "key", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "key", WindowedValue.of( - "v1", - new Instant(0), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v2", - new Instant(5), - Arrays.asList(window(5, 15)), - PaneInfo.NO_FIRING), + "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(15), - Arrays.asList(window(15, 25)), - PaneInfo.NO_FIRING)); + "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15))); + getOnlyElementInWindow(result, window(0, 15)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp())); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25))); + getOnlyElementInWindow(result, window(15, 25)); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp())); } @@ -331,39 +322,29 @@ public static void groupsElementsInMergedSessions( public static void combinesElementsPerSession( GroupAlsoByWindowsDoFnFactory gabwFactory, CombineFn combineFn) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))); - DoFnTester>>, KV> result = - runGABW(gabwFactory, windowingStrategy, "k", - WindowedValue.of( - 1L, - new Instant(0), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + List>> result = + runGABW( + gabwFactory, + windowingStrategy, + "k", + WindowedValue.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValue.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValue.of( - 2L, - new Instant(5), - Arrays.asList(window(5, 15)), - PaneInfo.NO_FIRING), - WindowedValue.of( - 4L, - new Instant(15), - Arrays.asList(window(15, 25)), - PaneInfo.NO_FIRING)); + 4L, new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); - TimestampedValue> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15))); + TimestampedValue> item0 = getOnlyElementInWindow(result, window(0, 15)); assertThat(item0.getValue().getKey(), equalTo("k")); assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L)))); assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp())); - TimestampedValue> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25))); + TimestampedValue> item1 = getOnlyElementInWindow(result, window(15, 25)); assertThat(item1.getValue().getKey(), equalTo("k")); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L)))); assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp())); @@ -371,176 +352,152 @@ public static void combinesElementsPerSession( /** * Tests that for a simple sequence of elements on the same key, the given GABW implementation - * correctly groups them according to fixed windows and also sets the output timestamp - * according to the policy {@link OutputTimeFns#outputAtEndOfWindow()}. + * correctly groups them according to fixed windows and also sets the output timestamp according + * to the policy {@link OutputTimeFns#outputAtEndOfWindow()}. */ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) - .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "key", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "key", WindowedValue.of( - "v1", - new Instant(1), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v2", - new Instant(2), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(13), - Arrays.asList(window(10, 20)), - PaneInfo.NO_FIRING)); + "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10))); + getOnlyElementInWindow(result, window(0, 10)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp())); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20))); + getOnlyElementInWindow(result, window(10, 20)); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp())); } /** * Tests that for a simple sequence of elements on the same key, the given GABW implementation - * correctly groups them according to fixed windows and also sets the output timestamp - * according to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}. + * correctly groups them according to fixed windows and also sets the output timestamp according + * to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}. */ public static void groupsElementsIntoFixedWindowsWithLatestTimestamp( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) - .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp()); + .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp()); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "k", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "k", WindowedValue.of( - "v1", - new Instant(1), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v2", - new Instant(2), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(13), - Arrays.asList(window(10, 20)), - PaneInfo.NO_FIRING)); + "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10))); + getOnlyElementInWindow(result, window(0, 10)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(new Instant(2))); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20))); + getOnlyElementInWindow(result, window(10, 20)); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(new Instant(13))); } /** - * Tests that the given GABW implementation correctly groups elements into merged sessions - * with output timestamps at the end of the merged window. + * Tests that the given GABW implementation correctly groups elements into merged sessions with + * output timestamps at the end of the merged window. */ public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "k", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "k", WindowedValue.of( - "v1", - new Instant(0), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v2", - new Instant(5), - Arrays.asList(window(5, 15)), - PaneInfo.NO_FIRING), + "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(15), - Arrays.asList(window(15, 25)), - PaneInfo.NO_FIRING)); + "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15))); + getOnlyElementInWindow(result, window(0, 15)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp())); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25))); + getOnlyElementInWindow(result, window(15, 25)); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp())); } /** - * Tests that the given GABW implementation correctly groups elements into merged sessions - * with output timestamps at the end of the merged window. + * Tests that the given GABW implementation correctly groups elements into merged sessions with + * output timestamps at the end of the merged window. */ public static void groupsElementsInMergedSessionsWithLatestTimestamp( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp()); BoundedWindow unmergedWindow = window(15, 25); - DoFnTester>>, KV>> result = - runGABW(gabwFactory, windowingStrategy, "k", + List>>> result = + runGABW( + gabwFactory, + windowingStrategy, + "k", WindowedValue.of( - "v1", - new Instant(0), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), + "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( - "v2", - new Instant(5), - Arrays.asList(window(5, 15)), - PaneInfo.NO_FIRING), + "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValue.of( - "v3", - new Instant(15), - Arrays.asList(unmergedWindow), - PaneInfo.NO_FIRING)); + "v3", new Instant(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); BoundedWindow mergedWindow = window(0, 15); TimestampedValue>> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(mergedWindow)); + getOnlyElementInWindow(result, mergedWindow); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(new Instant(5))); TimestampedValue>> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(unmergedWindow)); + getOnlyElementInWindow(result, unmergedWindow); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(new Instant(15))); } @@ -552,81 +509,66 @@ public static void groupsElementsInMergedSessionsWithLatestTimestamp( public static void combinesElementsPerSessionWithEndOfWindowTimestamp( GroupAlsoByWindowsDoFnFactory gabwFactory, CombineFn combineFn) - throws Exception { + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) - .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); BoundedWindow secondWindow = window(15, 25); - DoFnTester> result = - runGABW(gabwFactory, windowingStrategy, "k", - WindowedValue.of( - 1L, - new Instant(0), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), - WindowedValue.of( - 2L, - new Instant(5), - Arrays.asList(window(5, 15)), - PaneInfo.NO_FIRING), - WindowedValue.of( - 4L, - new Instant(15), - Arrays.asList(secondWindow), - PaneInfo.NO_FIRING)); + List>> result = + runGABW( + gabwFactory, + windowingStrategy, + "k", + WindowedValue.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValue.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValue.of(4L, new Instant(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); - assertThat(result.peekOutputElements(), hasSize(2)); + assertThat(result, hasSize(2)); BoundedWindow firstResultWindow = window(0, 15); - TimestampedValue> item0 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(firstResultWindow)); + TimestampedValue> item0 = getOnlyElementInWindow(result, firstResultWindow); assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L)))); assertThat(item0.getTimestamp(), equalTo(firstResultWindow.maxTimestamp())); - TimestampedValue> item1 = - Iterables.getOnlyElement(result.peekOutputElementsInWindow(secondWindow)); + TimestampedValue> item1 = getOnlyElementInWindow(result, secondWindow); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L)))); - assertThat(item1.getTimestamp(), - equalTo(secondWindow.maxTimestamp())); + assertThat(item1.getTimestamp(), equalTo(secondWindow.maxTimestamp())); } @SafeVarargs private static - DoFnTester>>, KV> runGABW( - GroupAlsoByWindowsDoFnFactory gabwFactory, - WindowingStrategy windowingStrategy, - K key, - WindowedValue... values) throws Exception { + List>> runGABW( + GroupAlsoByWindowsDoFnFactory gabwFactory, + WindowingStrategy windowingStrategy, + K key, + WindowedValue... values) + throws Exception { return runGABW(gabwFactory, windowingStrategy, key, Arrays.asList(values)); } private static - DoFnTester>>, KV> runGABW( - GroupAlsoByWindowsDoFnFactory gabwFactory, - WindowingStrategy windowingStrategy, - K key, - Collection> values) throws Exception { + List>> runGABW( + GroupAlsoByWindowsDoFnFactory gabwFactory, + WindowingStrategy windowingStrategy, + K key, + Collection> values) + throws Exception { final StateInternalsFactory stateInternalsCache = new CachingStateInternalsFactory(); - DoFnTester>>, KV> tester = - DoFnTester.of(gabwFactory.forStrategy(windowingStrategy, stateInternalsCache)); - - // Though we use a DoFnTester, the function itself is instantiated directly by the - // runner and should not be serialized; it may not even be serializable. - tester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); - tester.startBundle(); - tester.processElement(KV.>>of(key, values)); - tester.finishBundle(); + List>> output = + processElement( + gabwFactory.forStrategy(windowingStrategy, stateInternalsCache), + KV.>>of(key, values)); // Sanity check for corruption - for (KV elem : tester.peekOutputElements()) { - assertThat(elem.getKey(), equalTo(key)); + for (WindowedValue> value : output) { + assertThat(value.getValue().getKey(), equalTo(key)); } - return tester; + return output; } private static BoundedWindow window(long start, long end) { @@ -657,4 +599,158 @@ public StateInternals load(K key) throws Exception { return InMemoryStateInternals.forKey(key); } } + + private static + List>> processElement( + GroupAlsoByWindowsDoFn fn, + KV>> element) + throws Exception { + TestProcessContext c = new TestProcessContext<>(fn, element); + fn.processElement(c); + return c.getOutput(); + } + + private static TimestampedValue> getOnlyElementInWindow( + List>> output, final BoundedWindow window) { + WindowedValue> res = + Iterables.getOnlyElement( + Iterables.filter( + output, + new Predicate>>() { + @Override + public boolean apply(@Nullable WindowedValue> input) { + return input.getWindows().contains(window); + } + })); + return TimestampedValue.of(res.getValue(), res.getTimestamp()); + } + + /** + * A {@link GroupAlsoByWindowsDoFn.ProcessContext} providing just enough context for a {@link + * GroupAlsoByWindowsDoFn} - namely, information about the element and output via {@link + * WindowingInternals}, but no side inputs/outputs and no normal output. + */ + private static class TestProcessContext + extends GroupAlsoByWindowsDoFn.ProcessContext { + private final PipelineOptions options = PipelineOptionsFactory.create(); + private final KV>> element; + private final List>> output = new ArrayList<>(); + + private TestProcessContext( + GroupAlsoByWindowsDoFn fn, + KV>> element) { + fn.super(); + this.element = element; + } + + @Override + public KV>> element() { + return element; + } + + @Override + public Instant timestamp() { + return BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + @Override + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public T sideInput(PCollectionView view) { + throw new UnsupportedOperationException(); + } + + @Override + public WindowingInternals>>, KV> + windowingInternals() { + return new WindowingInternals>>, KV>() { + @Override + public void outputWindowedValue( + KV output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + TestProcessContext.this.output.add(WindowedValue.of(output, timestamp, windows, pane)); + } + + @Override + public void sideOutputWindowedValue( + TupleTag tag, + SideOutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + throw new UnsupportedOperationException(); + } + + @Override + public StateInternals stateInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public TimerInternals timerInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public Collection windows() { + return ImmutableList.of(GlobalWindow.INSTANCE); + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public void output(KV output) { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWithTimestamp(KV output, Instant timestamp) { + throw new UnsupportedOperationException(); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException(); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + throw new UnsupportedOperationException(); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { + throw new UnsupportedOperationException(); + } + + public List>> getOutput() { + return output; + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index 1a74ae740db3..6ee42e7a58ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn.Context; @@ -185,6 +186,7 @@ protected Aggregator createAggreg * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise, * returns {@code null}. */ + @Nullable public static DoFn getDoFn(OldDoFn fn) { if (fn instanceof SimpleDoFnAdapter) { return ((SimpleDoFnAdapter) fn).fn; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 17fa612aedb7..a9f93dd3eb14 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -23,10 +23,10 @@ import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -36,6 +36,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ValueInSingleWindow; import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -43,7 +45,6 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; @@ -86,7 +87,8 @@ public class DoFnTester implements AutoCloseable { */ @SuppressWarnings("unchecked") public static DoFnTester of(DoFn fn) { - return new DoFnTester<>(DoFnAdapters.toOldDoFn(fn)); + checkNotNull(fn, "fn can't be null"); + return new DoFnTester<>(fn); } /** @@ -96,9 +98,11 @@ public static DoFnTester of(DoFn DoFnTester + @Deprecated + public static DoFnTester of(OldDoFn fn) { - return new DoFnTester<>(fn); + checkNotNull(fn, "fn can't be null"); + return new DoFnTester<>(fn.toDoFn()); } /** @@ -238,7 +242,7 @@ public void startBundle() throws Exception { stateInternals = InMemoryStateInternals.forKey(new Object()); timerInternals = new InMemoryTimerInternals(); try { - fn.startBundle(context); + fnInvoker.invokeStartBundle(context); } catch (UserCodeException e) { unwrapUserCodeException(e); } @@ -271,8 +275,8 @@ public void processElement(InputT element) throws Exception { } /** - * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a - * context where {@link OldDoFn.ProcessContext#element} returns the + * Calls {@link DoFn.ProcessElement} on the {@code DoFn} under test, in a + * context where {@link DoFn.ProcessContext#element} returns the * given element and timestamp. * *

    Will call {@link #startBundle} automatically, if it hasn't @@ -286,7 +290,13 @@ public void processTimestampedElement(TimestampedValue element) throws E startBundle(); } try { - fn.processElement(createProcessContext(element)); + final TestProcessContext processContext = createProcessContext(element); + fnInvoker.invokeProcessElement(new DoFnInvoker.FakeArgumentProvider() { + @Override + public DoFn.ProcessContext processContext(DoFn doFn) { + return processContext; + } + }); } catch (UserCodeException e) { unwrapUserCodeException(e); } @@ -308,13 +318,14 @@ public void finishBundle() throws Exception { "Must be inside bundle to call finishBundle, but was: %s", state); try { - fn.finishBundle(createContext(fn)); + fnInvoker.invokeFinishBundle(createContext(fn)); } catch (UserCodeException e) { unwrapUserCodeException(e); } if (cloningBehavior == CloningBehavior.CLONE_PER_BUNDLE) { - fn.teardown(); + fnInvoker.invokeTeardown(); fn = null; + fnInvoker = null; state = State.UNINITIALIZED; } else { state = State.BUNDLE_FINISHED; @@ -532,11 +543,11 @@ public TupleTag getMainOutputTag() { return mainOutputTag; } - private TestContext createContext(OldDoFn fn) { + private TestContext createContext(DoFn fn) { return new TestContext(); } - private class TestContext extends OldDoFn.Context { + private class TestContext extends DoFn.Context { TestContext() { fn.super(); } @@ -557,7 +568,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { } @Override - protected Aggregator createAggregatorInternal( + protected Aggregator createAggregator( final String name, final CombineFn combiner) { return aggregator(name, combiner); } @@ -624,7 +635,7 @@ private TestProcessContext createProcessContext(TimestampedValue elem) { elem.getValue(), elem.getTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } - private class TestProcessContext extends OldDoFn.ProcessContext { + private class TestProcessContext extends DoFn.ProcessContext { private final TestContext context; private final ValueInSingleWindow element; @@ -644,7 +655,9 @@ public T sideInput(PCollectionView view) { Map viewValues = sideInputs.get(view); if (viewValues != null) { BoundedWindow sideInputWindow = - view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(window()); + view.getWindowingStrategyInternal() + .getWindowFn() + .getSideInputWindow(element.getWindow()); @SuppressWarnings("unchecked") T windowValue = (T) viewValues.get(sideInputWindow); if (windowValue != null) { @@ -659,73 +672,11 @@ public Instant timestamp() { return element.getTimestamp(); } - @Override - public BoundedWindow window() { - return element.getWindow(); - } - @Override public PaneInfo pane() { return element.getPane(); } - @Override - public WindowingInternals windowingInternals() { - return new WindowingInternals() { - @Override - public StateInternals stateInternals() { - return stateInternals; - } - - @Override - public void outputWindowedValue( - OutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - for (BoundedWindow window : windows) { - context.noteOutput( - mainOutputTag, ValueInSingleWindow.of(output, timestamp, window, pane)); - } - } - - @Override - public void sideOutputWindowedValue( - TupleTag tag, - SideOutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - for (BoundedWindow window : windows) { - context.noteOutput( - tag, ValueInSingleWindow.of(output, timestamp, window, pane)); - } - } - - @Override - public TimerInternals timerInternals() { - return timerInternals; - } - - @Override - public Collection windows() { - return Collections.singleton(element.getWindow()); - } - - @Override - public PaneInfo pane() { - return element.getPane(); - } - - @Override - public T sideInput( - PCollectionView view, BoundedWindow sideInputWindow) { - throw new UnsupportedOperationException( - "SideInput from WindowingInternals is not supported in in the context of DoFnTester"); - } - }; - } - @Override public PipelineOptions getPipelineOptions() { return context.getPipelineOptions(); @@ -753,10 +704,10 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - protected Aggregator createAggregatorInternal( + protected Aggregator createAggregator( String name, CombineFn combiner) { throw new IllegalStateException("Aggregators should not be created within ProcessContext. " - + "Instead, create an aggregator at OldDoFn construction time with" + + "Instead, create an aggregator at DoFn construction time with" + " createAggregator, and ensure they are set up by the time startBundle is" + " called with setupDelegateAggregators."); } @@ -768,8 +719,9 @@ public void close() throws Exception { finishBundle(); } if (state == State.BUNDLE_FINISHED) { - fn.teardown(); + fnInvoker.invokeTeardown(); fn = null; + fnInvoker = null; } state = State.TORN_DOWN; } @@ -786,8 +738,8 @@ private enum State { private final PipelineOptions options = PipelineOptionsFactory.create(); - /** The original {@link OldDoFn} under test. */ - private final OldDoFn origFn; + /** The original {@link DoFn} under test. */ + private final DoFn origFn; /** * Whether to clone the original {@link DoFn} or just use it as-is. @@ -805,8 +757,9 @@ private enum State { /** The output tags used by the {@link DoFn} under test. */ private TupleTag mainOutputTag = new TupleTag<>(); - /** The original OldDoFn under test, if started. */ - OldDoFn fn; + /** The original DoFn under test, if started. */ + private DoFn fn; + private DoFnInvoker fnInvoker; /** The outputs from the {@link DoFn} under test. */ private Map, List>> outputs; @@ -817,7 +770,7 @@ private enum State { /** The state of processing of the {@link DoFn} under test. */ private State state = State.UNINITIALIZED; - private DoFnTester(OldDoFn origFn) { + private DoFnTester(DoFn origFn) { this.origFn = origFn; } @@ -828,12 +781,13 @@ private void initializeState() throws Exception { if (cloningBehavior.equals(CloningBehavior.DO_NOT_CLONE)) { fn = origFn; } else { - fn = (OldDoFn) + fn = (DoFn) SerializableUtils.deserializeFromByteArray( SerializableUtils.serializeToByteArray(origFn), origFn.toString()); } - fn.setup(); + fnInvoker = DoFnInvokers.invokerFor(fn); + fnInvoker.invokeSetup(); outputs = new HashMap<>(); accumulators = new HashMap<>(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index 4ad7dad30e95..50a7082cdad5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -18,9 +18,6 @@ package org.apache.beam.sdk.transforms.reflect; import java.io.Serializable; -import java.lang.reflect.Constructor; -import java.util.LinkedHashMap; -import java.util.Map; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.transforms.DoFn; @@ -45,14 +42,6 @@ public static DoFnInvoker invokerFor( return ByteBuddyDoFnInvokerFactory.only().newByteBuddyInvoker(fn); } - /** - * A cache of constructors of generated {@link DoFnInvoker} classes, keyed by {@link DoFn} class. - * Needed because generating an invoker class is expensive, and to avoid generating an excessive - * number of classes consuming PermGen memory. - */ - private final Map, Constructor> byteBuddyInvokerConstructorCache = - new LinkedHashMap<>(); - private DoFnInvokers() {} /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index ac76b2e515b5..ff8a9bc08173 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -350,14 +350,14 @@ public void fnWithSideInputExplicit() throws Exception { } } - private static class SideInputDoFn extends OldDoFn { + private static class SideInputDoFn extends DoFn { private final PCollectionView value; private SideInputDoFn(PCollectionView value) { this.value = value; } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { c.output(c.sideInput(value)); } From 78ac009be743a2e053580e9966f841174b636e88 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 2 Dec 2016 11:39:48 -0800 Subject: [PATCH 067/279] Supports window parameter in DoFnTester Also prohibits other parameters, and prohibits output from bundle methods (whereas previously it was silently dropped). --- .../beam/sdk/transforms/DoFnTester.java | 166 +++++++++++++----- .../beam/sdk/transforms/DoFnTesterTest.java | 34 ++++ 2 files changed, 158 insertions(+), 42 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index a9f93dd3eb14..7c1abef250cf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -38,13 +38,18 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; 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.SerializableUtils; +import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; @@ -84,6 +89,9 @@ public class DoFnTester implements AutoCloseable { /** * Returns a {@code DoFnTester} supporting unit-testing of the given * {@link DoFn}. By default, uses {@link CloningBehavior#CLONE_ONCE}. + * + *

    The only supported extra parameter of the {@link DoFn.ProcessElement} method is + * {@link BoundedWindow}. */ @SuppressWarnings("unchecked") public static DoFnTester of(DoFn fn) { @@ -236,7 +244,7 @@ public void startBundle() throws Exception { if (state == State.UNINITIALIZED) { initializeState(); } - TestContext context = createContext(fn); + TestContext context = new TestContext(); context.setupDelegateAggregators(); // State and timer internals are per-bundle. stateInternals = InMemoryStateInternals.forKey(new Object()); @@ -262,7 +270,7 @@ private static void unwrapUserCodeException(UserCodeException e) throws Exceptio /** * Calls the {@link DoFn.ProcessElement} method on the {@link DoFn} under test, in a * context where {@link DoFn.ProcessContext#element} returns the - * given element. + * given element and the element is in the global window. * *

    Will call {@link #startBundle} automatically, if it hasn't * already been called. @@ -277,26 +285,86 @@ public void processElement(InputT element) throws Exception { /** * Calls {@link DoFn.ProcessElement} on the {@code DoFn} under test, in a * context where {@link DoFn.ProcessContext#element} returns the - * given element and timestamp. + * given element and timestamp and the element is in the global window. * *

    Will call {@link #startBundle} automatically, if it hasn't * already been called. - * - *

    If the input timestamp is {@literal null}, the minimum timestamp will be used. */ public void processTimestampedElement(TimestampedValue element) throws Exception { checkNotNull(element, "Timestamped element cannot be null"); + processWindowedElement( + element.getValue(), element.getTimestamp(), GlobalWindow.INSTANCE); + } + + /** + * Calls {@link DoFn.ProcessElement} on the {@code DoFn} under test, in a + * context where {@link DoFn.ProcessContext#element} returns the + * given element and timestamp and the element is in the given window. + * + *

    Will call {@link #startBundle} automatically, if it hasn't + * already been called. + */ + public void processWindowedElement( + InputT element, Instant timestamp, final BoundedWindow window) throws Exception { if (state != State.BUNDLE_STARTED) { startBundle(); } try { - final TestProcessContext processContext = createProcessContext(element); - fnInvoker.invokeProcessElement(new DoFnInvoker.FakeArgumentProvider() { - @Override - public DoFn.ProcessContext processContext(DoFn doFn) { - return processContext; - } - }); + final TestProcessContext processContext = + new TestProcessContext( + ValueInSingleWindow.of(element, timestamp, window, PaneInfo.NO_FIRING)); + fnInvoker.invokeProcessElement( + new DoFnInvoker.ArgumentProvider() { + @Override + public BoundedWindow window() { + return window; + } + + @Override + public DoFn.Context context(DoFn doFn) { + throw new UnsupportedOperationException( + "Not expected to access DoFn.Context from @ProcessElement"); + } + + @Override + public DoFn.ProcessContext processContext(DoFn doFn) { + return processContext; + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException( + "Not expected to access InputProvider from DoFnTester"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException( + "Not expected to access OutputReceiver from DoFnTester"); + } + + @Override + public WindowingInternals windowingInternals() { + throw new UnsupportedOperationException( + "Not expected to access WindowingInternals from a new DoFn"); + } + + @Override + public RestrictionTracker restrictionTracker() { + throw new UnsupportedOperationException( + "Not expected to access RestrictionTracker from a regular DoFn in DoFnTester"); + } + + @Override + public org.apache.beam.sdk.util.state.State state(String stateId) { + throw new UnsupportedOperationException("DoFnTester doesn't support state yet"); + } + + @Override + public Timer timer(String timerId) { + throw new UnsupportedOperationException("DoFnTester doesn't support timers yet"); + } + }); } catch (UserCodeException e) { unwrapUserCodeException(e); } @@ -318,7 +386,7 @@ public void finishBundle() throws Exception { "Must be inside bundle to call finishBundle, but was: %s", state); try { - fnInvoker.invokeFinishBundle(createContext(fn)); + fnInvoker.invokeFinishBundle(new TestContext()); } catch (UserCodeException e) { unwrapUserCodeException(e); } @@ -543,10 +611,6 @@ public TupleTag getMainOutputTag() { return mainOutputTag; } - private TestContext createContext(DoFn fn) { - return new TestContext(); - } - private class TestContext extends DoFn.Context { TestContext() { fn.super(); @@ -559,12 +623,27 @@ public PipelineOptions getPipelineOptions() { @Override public void output(OutputT output) { - sideOutput(mainOutputTag, output); + throwUnsupportedOutputFromBundleMethods(); } @Override public void outputWithTimestamp(OutputT output, Instant timestamp) { - sideOutputWithTimestamp(mainOutputTag, output, timestamp); + throwUnsupportedOutputFromBundleMethods(); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + throwUnsupportedOutputFromBundleMethods(); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throwUnsupportedOutputFromBundleMethods(); + } + + private void throwUnsupportedOutputFromBundleMethods() { + throw new UnsupportedOperationException( + "DoFnTester doesn't support output from bundle methods"); } @Override @@ -613,26 +692,6 @@ public String getName() { } return aggregator; } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - - } - - @Override - public void sideOutput(TupleTag tag, T output) { - sideOutputWithTimestamp(tag, output, BoundedWindow.TIMESTAMP_MIN_VALUE); - } - - public void noteOutput(TupleTag tag, ValueInSingleWindow output) { - getMutableOutput(tag).add(output); - } - } - - private TestProcessContext createProcessContext(TimestampedValue elem) { - return new TestProcessContext( - ValueInSingleWindow.of( - elem.getValue(), elem.getTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } private class TestProcessContext extends DoFn.ProcessContext { @@ -641,7 +700,7 @@ private class TestProcessContext extends DoFn.ProcessContext { private TestProcessContext(ValueInSingleWindow element) { fn.super(); - this.context = createContext(fn); + this.context = new TestContext(); this.element = element; } @@ -699,8 +758,8 @@ public void sideOutput(TupleTag tag, T output) { @Override public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.noteOutput( - tag, ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane())); + getMutableOutput(tag) + .add(ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane())); } @Override @@ -772,6 +831,29 @@ private enum State { private DoFnTester(DoFn origFn) { this.origFn = origFn; + DoFnSignature signature = DoFnSignatures.signatureForDoFn(origFn); + for (DoFnSignature.Parameter param : signature.processElement().extraParameters()) { + param.match( + new DoFnSignature.Parameter.Cases.WithDefault() { + @Override + public Void dispatch(DoFnSignature.Parameter.ProcessContextParameter p) { + // ProcessContext parameter is obviously supported. + return null; + } + + @Override + public Void dispatch(DoFnSignature.Parameter.WindowParameter p) { + // We also support the BoundedWindow parameter. + return null; + } + + @Override + protected Void dispatchDefault(DoFnSignature.Parameter p) { + throw new UnsupportedOperationException( + "Parameter " + p + " not supported by DoFnTester"); + } + }); + } } @SuppressWarnings("unchecked") diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index ff8a9bc08173..b47465eab436 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -30,13 +30,16 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.TestPipeline; +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.util.PCollectionViews; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.hamcrest.Matchers; +import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -350,6 +353,37 @@ public void fnWithSideInputExplicit() throws Exception { } } + @Test + public void testSupportsWindowParameter() throws Exception { + Instant now = Instant.now(); + try (DoFnTester> tester = + DoFnTester.of(new DoFnWithWindowParameter())) { + BoundedWindow firstWindow = new IntervalWindow(now, now.plus(Duration.standardMinutes(1))); + tester.processWindowedElement(1, now, firstWindow); + tester.processWindowedElement(2, now, firstWindow); + BoundedWindow secondWindow = new IntervalWindow(now, now.plus(Duration.standardMinutes(4))); + tester.processWindowedElement(3, now, secondWindow); + tester.finishBundle(); + + assertThat( + tester.peekOutputElementsInWindow(firstWindow), + containsInAnyOrder( + TimestampedValue.of(KV.of(1, firstWindow), now), + TimestampedValue.of(KV.of(2, firstWindow), now))); + assertThat( + tester.peekOutputElementsInWindow(secondWindow), + containsInAnyOrder( + TimestampedValue.of(KV.of(3, secondWindow), now))); + } + } + + private static class DoFnWithWindowParameter extends DoFn> { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) { + c.output(KV.of(c.element(), window)); + } + } + private static class SideInputDoFn extends DoFn { private final PCollectionView value; From 86173a839f57cf7ed45566b380e557cf1defcba9 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 2 Dec 2016 11:44:02 -0800 Subject: [PATCH 068/279] Removes DoFnTester.of(OldDoFn) --- .../apache/beam/sdk/transforms/DoFnTester.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 7c1abef250cf..9f32aec9768d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -23,7 +23,6 @@ import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Arrays; @@ -99,20 +98,6 @@ public static DoFnTester of(DoFn(fn); } - /** - * Returns a {@code DoFnTester} supporting unit-testing of the given - * {@link OldDoFn}. - * - * @see #of(DoFn) - */ - @SuppressWarnings("unchecked") - @Deprecated - public static DoFnTester - of(OldDoFn fn) { - checkNotNull(fn, "fn can't be null"); - return new DoFnTester<>(fn.toDoFn()); - } - /** * Registers the tuple of values of the side input {@link PCollectionView}s to * pass to the {@link DoFn} under test. From 9a038c4f3404a3707eca29c5e898014df7fafbf4 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Wed, 30 Nov 2016 14:06:59 -0800 Subject: [PATCH 069/279] Add TextIO.Write support for runtime-valued output prefix * Updates to TextIO * Updates for FileBasedSink to support this change * Updates to other FileBasedSinks that do not yet support runtime values but need to be aware that values are now ValueProvider instead of String --- .../org/apache/beam/sdk/io/FileBasedSink.java | 22 +++++++++------ .../java/org/apache/beam/sdk/io/TextIO.java | 28 +++++++++++++++---- .../java/org/apache/beam/sdk/io/XmlSink.java | 4 +-- .../org/apache/beam/sdk/io/XmlSinkTest.java | 6 ++-- 4 files changed, 42 insertions(+), 18 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index 5375b90b68f1..1396ab65ab9f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -41,6 +41,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; @@ -135,7 +137,7 @@ public String getMimeType() { /** * Base filename for final output files. */ - protected final String baseOutputFilename; + protected final ValueProvider baseOutputFilename; /** * The extension to be used for the final output files. @@ -162,7 +164,8 @@ public FileBasedSink(String baseOutputFilename, String extension) { */ public FileBasedSink(String baseOutputFilename, String extension, WritableByteChannelFactory writableByteChannelFactory) { - this(baseOutputFilename, extension, ShardNameTemplate.INDEX_OF_MAX, writableByteChannelFactory); + this(StaticValueProvider.of(baseOutputFilename), extension, + ShardNameTemplate.INDEX_OF_MAX, writableByteChannelFactory); } /** @@ -173,7 +176,8 @@ public FileBasedSink(String baseOutputFilename, String extension, *

    See {@link ShardNameTemplate} for a description of file naming templates. */ public FileBasedSink(String baseOutputFilename, String extension, String fileNamingTemplate) { - this(baseOutputFilename, extension, fileNamingTemplate, CompressionType.UNCOMPRESSED); + this(StaticValueProvider.of(baseOutputFilename), extension, fileNamingTemplate, + CompressionType.UNCOMPRESSED); } /** @@ -182,8 +186,8 @@ public FileBasedSink(String baseOutputFilename, String extension, String fileNam * *

    See {@link ShardNameTemplate} for a description of file naming templates. */ - public FileBasedSink(String baseOutputFilename, String extension, String fileNamingTemplate, - WritableByteChannelFactory writableByteChannelFactory) { + public FileBasedSink(ValueProvider baseOutputFilename, String extension, + String fileNamingTemplate, WritableByteChannelFactory writableByteChannelFactory) { this.writableByteChannelFactory = writableByteChannelFactory; this.baseOutputFilename = baseOutputFilename; if (!isNullOrEmpty(writableByteChannelFactory.getFilenameSuffix())) { @@ -198,7 +202,7 @@ public FileBasedSink(String baseOutputFilename, String extension, String fileNam * Returns the base output filename for this file based sink. */ public String getBaseOutputFilename() { - return baseOutputFilename; + return baseOutputFilename.get(); } @Override @@ -216,7 +220,9 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); String fileNamePattern = String.format("%s%s%s", - baseOutputFilename, fileNamingTemplate, getFileExtension(extension)); + baseOutputFilename.isAccessible() + ? baseOutputFilename.get() : baseOutputFilename.toString(), + fileNamingTemplate, getFileExtension(extension)); builder.add(DisplayData.item("fileNamePattern", fileNamePattern) .withLabel("File Name Pattern")); } @@ -420,7 +426,7 @@ protected final List copyToOutputFiles(List filenames, PipelineO protected final List generateDestinationFilenames(int numFiles) { List destFilenames = new ArrayList<>(); String extension = getSink().extension; - String baseOutputFilename = getSink().baseOutputFilename; + String baseOutputFilename = getSink().baseOutputFilename.get(); String fileNamingTemplate = getSink().fileNamingTemplate; String suffix = getFileExtension(extension); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 84c24ea859c7..e967a27d3fd1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -403,6 +403,13 @@ public static Bound to(String prefix) { return new Bound<>(DEFAULT_TEXT_CODER).to(prefix); } + /** + * Like {@link #to(String)}, but with a {@link ValueProvider}. + */ + public static Bound to(ValueProvider prefix) { + return new Bound<>(DEFAULT_TEXT_CODER).to(prefix); + } + /** * Returns a transform for writing to text files that appends the specified suffix * to the created files. @@ -521,7 +528,7 @@ public static class Bound extends PTransform, PDone> { private static final String DEFAULT_SHARD_TEMPLATE = ShardNameTemplate.INDEX_OF_MAX; /** The prefix of each file written, combined with suffix and shardTemplate. */ - @Nullable private final String filenamePrefix; + private final ValueProvider filenamePrefix; /** The suffix of each file written, combined with prefix and shardTemplate. */ private final String filenameSuffix; @@ -554,7 +561,7 @@ public static class Bound extends PTransform, PDone> { FileBasedSink.CompressionType.UNCOMPRESSED); } - private Bound(String name, String filenamePrefix, String filenameSuffix, + private Bound(String name, ValueProvider filenamePrefix, String filenameSuffix, @Nullable String header, @Nullable String footer, Coder coder, int numShards, String shardTemplate, boolean validate, WritableByteChannelFactory writableByteChannelFactory) { @@ -581,6 +588,15 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, */ public Bound to(String filenamePrefix) { validateOutputComponent(filenamePrefix); + return new Bound<>(name, StaticValueProvider.of(filenamePrefix), filenameSuffix, + header, footer, coder, numShards, shardTemplate, validate, + writableByteChannelFactory); + } + + /** + * Like {@link #to(String)}, but with a {@link ValueProvider}. + */ + public Bound to(ValueProvider filenamePrefix) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate, writableByteChannelFactory); } @@ -745,8 +761,10 @@ public PDone apply(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); + String prefixString = filenamePrefix.isAccessible() + ? filenamePrefix.get() : filenamePrefix.toString(); builder - .addIfNotNull(DisplayData.item("filePrefix", filenamePrefix) + .addIfNotNull(DisplayData.item("filePrefix", prefixString) .withLabel("Output File Prefix")) .addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix) .withLabel("Output Fix Suffix"), "") @@ -779,7 +797,7 @@ protected Coder getDefaultOutputCoder() { } public String getFilenamePrefix() { - return filenamePrefix; + return filenamePrefix.get(); } public String getShardTemplate() { @@ -1101,7 +1119,7 @@ static class TextSink extends FileBasedSink { @VisibleForTesting TextSink( - String baseOutputFilename, String extension, + ValueProvider baseOutputFilename, String extension, @Nullable String header, @Nullable String footer, String fileNameTemplate, Coder coder, WritableByteChannelFactory writableByteChannelFactory) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java index 983eed2468a5..0f25aeaf523c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java @@ -176,7 +176,7 @@ private Bound(Class classToBind, String rootElementName, String baseOutputFil *

    The specified class must be able to be used to create a JAXB context. */ public Bound ofRecordClass(Class classToBind) { - return new Bound<>(classToBind, rootElementName, baseOutputFilename); + return new Bound<>(classToBind, rootElementName, baseOutputFilename.get()); } /** @@ -194,7 +194,7 @@ public Bound toFilenamePrefix(String baseOutputFilename) { * supplied name. */ public Bound withRootElement(String rootElementName) { - return new Bound<>(classToBind, rootElementName, baseOutputFilename); + return new Bound<>(classToBind, rootElementName, baseOutputFilename.get()); } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java index 400b04a6c70b..f9a9655e1646 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java @@ -93,7 +93,7 @@ public void testBuildXmlSink() { .withRootElement(testRootElement); assertEquals(testClass, sink.classToBind); assertEquals(testRootElement, sink.rootElementName); - assertEquals(testFilePrefix, sink.baseOutputFilename); + assertEquals(testFilePrefix, sink.baseOutputFilename.get()); } /** @@ -105,7 +105,7 @@ public void testBuildXmlSinkDirect() { XmlSink.writeOf(Bird.class, testRootElement, testFilePrefix); assertEquals(testClass, sink.classToBind); assertEquals(testRootElement, sink.rootElementName); - assertEquals(testFilePrefix, sink.baseOutputFilename); + assertEquals(testFilePrefix, sink.baseOutputFilename.get()); } /** @@ -142,7 +142,7 @@ public void testCreateWriteOperations() { XmlSink.writeOf(testClass, testRootElement, testFilePrefix); XmlWriteOperation writeOp = sink.createWriteOperation(options); assertEquals(testClass, writeOp.getSink().classToBind); - assertEquals(testFilePrefix, writeOp.getSink().baseOutputFilename); + assertEquals(testFilePrefix, writeOp.getSink().baseOutputFilename.get()); assertEquals(testRootElement, writeOp.getSink().rootElementName); assertEquals(XmlSink.XML_EXTENSION, writeOp.getSink().extension); Path outputPath = new File(testFilePrefix).toPath(); From ad9ca455218f6dda32e31ee97fe721e8b4ad6c2a Mon Sep 17 00:00:00 2001 From: Jason Kuster Date: Mon, 14 Nov 2016 15:35:40 -0800 Subject: [PATCH 070/279] Initial commit of jobs Signed-off-by: Jason Kuster --- .jenkins/common_job_properties.groovy | 166 ++++++++++++++++++ ...b_beam_PostCommit_Java_MavenInstall.groovy | 42 +++++ ...mit_Java_RunnableOnService_Dataflow.groovy | 39 ++++ ...Commit_Java_RunnableOnService_Flink.groovy | 38 ++++ ...mit_Java_RunnableOnService_Gearpump.groovy | 41 +++++ ...Commit_Java_RunnableOnService_Spark.groovy | 38 ++++ .../job_beam_PostCommit_Python_Verify.groovy | 37 ++++ ...ob_beam_PreCommit_Java_MavenInstall.groovy | 42 +++++ .../job_beam_Release_NightlySnapshot.groovy | 46 +++++ .jenkins/job_seed.groovy | 47 +++++ 10 files changed, 536 insertions(+) create mode 100644 .jenkins/common_job_properties.groovy create mode 100644 .jenkins/job_beam_PostCommit_Java_MavenInstall.groovy create mode 100644 .jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy create mode 100644 .jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy create mode 100644 .jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy create mode 100644 .jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy create mode 100644 .jenkins/job_beam_PostCommit_Python_Verify.groovy create mode 100644 .jenkins/job_beam_PreCommit_Java_MavenInstall.groovy create mode 100644 .jenkins/job_beam_Release_NightlySnapshot.groovy create mode 100644 .jenkins/job_seed.groovy diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy new file mode 100644 index 000000000000..f3a8a07952a0 --- /dev/null +++ b/.jenkins/common_job_properties.groovy @@ -0,0 +1,166 @@ +/* + * 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. + */ + +// Contains functions that help build Jenkins projects. Functions typically set +// common properties that are shared among all Jenkins projects. +class common_job_properties { + + // Sets common top-level job properties. + static def setTopLevelJobProperties(def context, + def default_branch = 'master', + def default_timeout = 100) { + // GitHub project. + context.properties { + githubProjectUrl('https://github.com/apache/incubator-beam/') + } + + // Set JDK version. + context.jdk('JDK 1.8 (latest)') + + // Restrict this project to run only on Jenkins executors dedicated to the + // Apache Beam project. + context.label('beam') + + // Discard old builds. Build records are only kept up to this number of days. + context.logRotator { + daysToKeep(14) + } + + // Source code management. + context.scm { + git { + remote { + url('https://github.com/apache/incubator-beam.git') + refspec('+refs/heads/*:refs/remotes/origin/* ' + + '+refs/pull/*:refs/remotes/origin/pr/*') + } + branch('${sha1}') + extensions { + cleanAfterCheckout() + } + } + } + + context.parameters { + // This is a recommended setup if you want to run the job manually. The + // ${sha1} parameter needs to be provided, and defaults to the main branch. + stringParam( + 'sha1', + default_branch, + 'Commit id or refname (eg: origin/pr/9/head) you want to build.') + } + + context.wrappers { + // Abort the build if it's stuck for more minutes than specified. + timeout { + absolute(default_timeout) + abortBuild() + } + + // Set SPARK_LOCAL_IP for spark tests. + environmentVariables { + env('SPARK_LOCAL_IP', '127.0.0.1') + } + } + } + + // Sets the pull request build trigger. + static def setPullRequestBuildTrigger(def context, + def commitStatusContext, + def successComment = '--none--') { + context.triggers { + githubPullRequest { + admins(['asfbot']) + useGitHubHooks() + orgWhitelist(['apache']) + allowMembersOfWhitelistedOrgsAsAdmin() + permitAll() + + extensions { + commitStatus { + // This is the name that will show up in the GitHub pull request UI + // for this Jenkins project. + delegate.context(commitStatusContext) + } + + /* + This section is disabled, because of jenkinsci/ghprb-plugin#417 issue. + For the time being, an equivalent configure section below is added. + + // Comment messages after build completes. + buildStatus { + completedStatus('SUCCESS', successComment) + completedStatus('FAILURE', '--none--') + completedStatus('ERROR', '--none--') + } + */ + } + } + } + + // Comment messages after build completes. + context.configure { + def messages = it / triggers / 'org.jenkinsci.plugins.ghprb.GhprbTrigger' / extensions / 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildStatus' / messages + messages << 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildResultMessage' { + message(successComment) + result('SUCCESS') + } + messages << 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildResultMessage' { + message('--none--') + result('ERROR') + } + messages << 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildResultMessage' { + message('--none--') + result('FAILURE') + } + } + } + + // Sets common config for Maven jobs. + static def setMavenConfig(def context) { + context.mavenInstallation('Maven 3.3.3') + context.rootPOM('pom.xml') + // Use a repository local to the workspace for better isolation of jobs. + context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE) + } + + // Sets common config for PreCommit jobs. + static def setPreCommit(def context, comment) { + // Set pull request build trigger. + setPullRequestBuildTrigger(context, comment) + } + + // Sets common config for PostCommit jobs. + static def setPostCommit(def context, + def build_schedule = '0 */6 * * *', + def scm_schedule = '* * * * *', + def notify_address = 'commits@beam.incubator.apache.org') { + // Set build triggers + context.triggers { + // By default runs every 6 hours. + cron(build_schedule) + // Also polls SCM every minute. + scm(scm_schedule) + } + + context.publishers { + // Notify an email address for each failed build (defaults to commits@). + mailer(notify_address, false, true) + } + } +} diff --git a/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy b/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy new file mode 100644 index 000000000000..7fc6c3b68217 --- /dev/null +++ b/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy @@ -0,0 +1,42 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the Java postcommit tests, including the suite of integration +// tests. +mavenJob('beam_PostCommit_Java_MavenInstall') { + description('Runs postcommit tests on the Java SDK.') + + previousNames('beam_PostCommit_MavenVerify') + + // Execute concurrent builds if necessary. + concurrentBuild() + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit(delegate) + + // Maven goals for this job. + goals('-B -e -P release,dataflow-runner clean install -DskipITs=false -DintegrationTestPipelineOptions=\'[ "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-end-to-end-tests", "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner" ]\'') +} diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy new file mode 100644 index 000000000000..44136ade8f20 --- /dev/null +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy @@ -0,0 +1,39 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the suite of RunnableOnService tests against the Dataflow +// runner. +mavenJob('beam_PostCommit_Java_RunnableOnService_Dataflow') { + description('Runs the RunnableOnService suite on the Dataflow runner.') + + previousNames('beam_PostCommit_RunnableOnService_GoogleCloudDataflow') + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit(delegate) + + // Maven goals for this job. + goals('-B -e clean verify -am -pl runners/google-cloud-dataflow-java -DforkCount=0 -DrunnableOnServicePipelineOptions=\'[ "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner", "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-runnable-on-service-tests/" ]\'') +} diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy new file mode 100644 index 000000000000..5a5863f31e45 --- /dev/null +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy @@ -0,0 +1,38 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the suite of RunnableOnService tests against the Flink runner. +mavenJob('beam_PostCommit_Java_RunnableOnService_Flink') { + description('Runs the RunnableOnService suite on the Flink runner.') + + previousNames('beam_PostCommit_RunnableOnService_FlinkLocal') + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit(delegate) + + // Maven goals for this job. + goals('-B -e clean verify -am -pl runners/flink/runner -Plocal-runnable-on-service-tests -Prunnable-on-service-tests') +} diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy new file mode 100644 index 000000000000..f07bbc6f7a09 --- /dev/null +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy @@ -0,0 +1,41 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the suite of RunnableOnService tests against the Gearpump +// runner. +mavenJob('beam_PostCommit_Java_RunnableOnService_Gearpump') { + description('Runs the RunnableOnService suite on the Gearpump runner.') + + previousNames('beam_PostCommit_RunnableOnService_GearpumpLocal') + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate, 'gearpump-runner') + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Sets that this is a PostCommit job. + // 0 5 31 2 * will run on Feb 31 (i.e. never) according to job properties. + // This job triggers only on SCM. + common_job_properties.setPostCommit(delegate, '0 5 31 2 *') + + // Maven goals for this job. + goals('-B -e clean verify -am -pl runners/gearpump -DforkCount=0 -DrunnableOnServicePipelineOptions=\'[ "--runner=TestGearpumpRunner", "--streaming=false" ]\'') +} diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy new file mode 100644 index 000000000000..875c6e6488b0 --- /dev/null +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy @@ -0,0 +1,38 @@ +/* + * 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. + */ + +import common_job_properties + +// This job runs the suite of RunnableOnService tests against the Spark runner. +mavenJob('beam_PostCommit_Java_RunnableOnService_Spark') { + description('Runs the RunnableOnService suite on the Spark runner.') + + previousNames('beam_PostCommit_RunnableOnService_SparkLocal') + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit(delegate) + + // Maven goals for this job. + goals('-B -e clean verify -am -pl runners/spark -Prunnable-on-service-tests -Plocal-runnable-on-service-tests -Dspark.port.maxRetries=64 -Dspark.ui.enabled=false') +} diff --git a/.jenkins/job_beam_PostCommit_Python_Verify.groovy b/.jenkins/job_beam_PostCommit_Python_Verify.groovy new file mode 100644 index 000000000000..79e173b8c74d --- /dev/null +++ b/.jenkins/job_beam_PostCommit_Python_Verify.groovy @@ -0,0 +1,37 @@ +/* + * 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. + */ + +import common_job_properties + +// This job defines the Python postcommit tests. +job('beam_PostCommit_Python_Verify') { + description('Runs postcommit tests on the Python SDK.') + + previousNames('beam_PostCommit_PythonVerify') + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate, 'python-sdk') + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit(delegate, '0 3-22/6 * * *') + + // Execute shell command to test Python SDK. + steps { + shell('bash sdks/python/run_postcommit.sh') + } +} diff --git a/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy b/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy new file mode 100644 index 000000000000..8a8dea573f4d --- /dev/null +++ b/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy @@ -0,0 +1,42 @@ +/* + * 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. + */ + +import common_job_properties + +// This is the Java precommit which runs a maven install, and the current set +// of precommit tests. +mavenJob('beam_PreCommit_Java_MavenInstall') { + description('Runs an install of the current GitHub Pull Request.') + + previousNames('beam_PreCommit_MavenVerify') + + // Execute concurrent builds if necessary. + concurrentBuild() + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set Maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Sets that this is a PreCommit job. + common_job_properties.setPreCommit(delegate, 'Jenkins: Maven clean install') + + // Maven goals for this job. + goals('-B -e -Prelease,include-runners,jenkins-precommit,direct-runner,dataflow-runner,spark-runner,flink-runner,apex-runner help:effective-settings clean install') +} diff --git a/.jenkins/job_beam_Release_NightlySnapshot.groovy b/.jenkins/job_beam_Release_NightlySnapshot.groovy new file mode 100644 index 000000000000..496a0c5bf163 --- /dev/null +++ b/.jenkins/job_beam_Release_NightlySnapshot.groovy @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import common_job_properties + +// This is the nightly snapshot build -- we use this to deploy a daily snapshot +// to https://repository.apache.org/content/groups/snapshots/org/apache/beam. +// Runs the postsubmit suite before deploying. +mavenJob('beam_Release_NightlySnapshot') { + description('Runs a mvn clean deploy of the nightly snapshot.') + + // Execute concurrent builds if necessary. + concurrentBuild() + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set maven paramaters. + common_job_properties.setMavenConfig(delegate) + + // Set that this is a PostCommit job. + // Polls SCM on Feb 31st, i.e. never. + common_job_properties.setPostCommit( + delegate, + '0 7 * * *', + '0 5 31 2 *', + 'dev@beam.incubator.apache.org') + + // Maven goals for this job. + goals('-B -e clean deploy -P release,dataflow-runner -DskipITs=false -DintegrationTestPipelineOptions=\'[ "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-end-to-end-tests", "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner" ]\'') +} diff --git a/.jenkins/job_seed.groovy b/.jenkins/job_seed.groovy new file mode 100644 index 000000000000..e46e5a44ad62 --- /dev/null +++ b/.jenkins/job_seed.groovy @@ -0,0 +1,47 @@ +/* + * 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. + */ + +import common_job_properties + +// Defines the seed job, which creates or updates all other Jenkins projects. +job('beam_SeedJob_Main') { + description('Automatically configures all Apache Beam main repo Jenkins ' + + 'projects based on Jenkins DSL groovy files checked into the ' + + 'code repository.') + + // Set common parameters. + common_job_properties.setTopLevelJobProperties(delegate) + + // Set that this is a PostCommit job. + // Polls SCM on Feb 31st, i.e. never. + common_job_properties.setPostCommit( + delegate, + '0 6 * * *', + '0 5 31 2 *', + 'dev@beam.incubator.apache.org') + + steps { + dsl { + // A list or a glob of other groovy files to process. + external('.jenkins/job_*.groovy') + + // If a job is removed from the script, disable it (rather than deleting). + removeAction('DISABLE') + } + } +} From 5a997a1a5d5d977bb84af1737db1128df916de7a Mon Sep 17 00:00:00 2001 From: Neelesh Srinivas Salian Date: Fri, 2 Dec 2016 17:43:34 -0800 Subject: [PATCH 071/279] BEAM-1078: Changing the links from GCP to incubator-beam in the project --- .travis/README.md | 2 +- .../org/apache/beam/examples/complete/README.md | 14 +++++++------- .../org/apache/beam/examples/cookbook/README.md | 14 +++++++------- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/.travis/README.md b/.travis/README.md index e0c13f2c6fd1..536692d69569 100644 --- a/.travis/README.md +++ b/.travis/README.md @@ -19,5 +19,5 @@ # Travis Scripts -This directory contains scripts used for [Travis CI](https://travis-ci.org/GoogleCloudPlatform/DataflowJavaSDK) +This directory contains scripts used for [Travis CI](https://travis-ci.org/apache/incubator-beam/) testing. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md index b98be7a72396..b0b6f9d62423 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md @@ -22,34 +22,34 @@ This directory contains end-to-end example pipelines that perform complex data processing tasks. They include:

      -
    • AutoComplete +
    • AutoComplete — An example that computes the most popular hash tags for every prefix, which can be used for auto-completion. Demonstrates how to use the same pipeline in both streaming and batch, combiners, and composite transforms.
    • -
    • StreamingWordExtract +
    • StreamingWordExtract — A streaming pipeline example that inputs lines of text from a Cloud Pub/Sub topic, splits each line into individual words, capitalizes those words, and writes the output to a BigQuery table.
    • -
    • TfIdf +
    • TfIdf — An example that computes a basic TF-IDF search table for a directory or Cloud Storage prefix. Demonstrates joining data, side inputs, and logging.
    • -
    • TopWikipediaSessions +
    • TopWikipediaSessions — An example that reads Wikipedia edit data from Cloud Storage and computes the user with the longest string of edits separated by no more than an hour within each month. Demonstrates using Cloud Dataflow Windowing to perform time-based aggregations of data.
    • -
    • TrafficMaxLaneFlow +
    • TrafficMaxLaneFlow — A streaming Beam Example using BigQuery output in the traffic sensor domain. Demonstrates the Cloud Dataflow streaming runner, sliding windows, Cloud Pub/Sub topic ingestion, the use of the AvroCoder to encode a custom class, and custom Combine transforms.
    • -
    • TrafficRoutes +
    • TrafficRoutes — A streaming Beam Example using BigQuery output in the traffic sensor domain. Demonstrates the Cloud Dataflow streaming runner, GroupByKey, keyed state, sliding windows, and Cloud @@ -57,6 +57,6 @@ This directory contains end-to-end example pipelines that perform complex data p
    -See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples +See the [documentation](http://beam.incubator.apache.org/get-started/quickstart/) and the [Examples README](../../../../../../../../../README.md) for information about how to run these examples. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md index 2edd26b71f06..e7099556d484 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md @@ -24,12 +24,12 @@ commonly-used data analysis patterns that you would likely incorporate into a larger Dataflow pipeline. They include:
      -
    • BigQueryTornadoes +
    • BigQueryTornadoes — An example that reads the public samples of weather data from Google BigQuery, counts the number of tornadoes that occur in each month, and writes the results to BigQuery. Demonstrates reading/writing BigQuery, counting a PCollection, and user-defined PTransforms.
    • -
    • CombinePerKeyExamples +
    • CombinePerKeyExamples — An example that reads the public "Shakespeare" data, and for each word in the dataset that exceeds a given length, generates a string containing the list of play names in which that word appears. @@ -39,13 +39,13 @@ larger Dataflow pipeline. They include:
    • DatastoreWordCount — An example that shows you how to read from Google Cloud Datastore.
    • -
    • DeDupExample +
    • DistinctExample — An example that uses Shakespeare's plays as plain text files, and removes duplicate lines across all the files. Demonstrates the Distinct, TextIO.Read, and TextIO.Write transforms, and how to wire transforms together.
    • -
    • FilterExamples +
    • FilterExamples — An example that shows different approaches to filtering, including selection and projection. It also shows how to dynamically set parameters by defining and using new pipeline options, and use how to use a value derived @@ -53,14 +53,14 @@ larger Dataflow pipeline. They include: Options configuration, and using pipeline-derived data as a side input.
    • -
    • JoinExamples +
    • JoinExamples — An example that shows how to join two collections. It uses a sample of the GDELT "world event" data, joining the event action country code against a table that maps country codes to country names. Demonstrates the Join operation, and using multiple input sources.
    • -
    • MaxPerKeyExamples +
    • MaxPerKeyExamples — An example that reads the public samples of weather data from BigQuery, and finds the maximum temperature (mean_temp) for each month. Demonstrates the Max statistical combination transform, and how to @@ -68,6 +68,6 @@ larger Dataflow pipeline. They include:
    -See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples +See the [documentation](http://beam.incubator.apache.org/get-started/quickstart/) and the [Examples README](../../../../../../../../../README.md) for information about how to run these examples. From 8f712fd6291803bfcda312ad7c31cb5c811c6508 Mon Sep 17 00:00:00 2001 From: Neelesh Srinivas Salian Date: Sat, 3 Dec 2016 09:08:54 -0800 Subject: [PATCH 072/279] BEAM-1083: Removing the link for the DatastoreWordCount in the README --- .../src/main/java/org/apache/beam/examples/cookbook/README.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md index e7099556d484..105fb4bb29c2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md @@ -37,8 +37,6 @@ larger Dataflow pipeline. They include: transform, which lets you combine the values in a key-grouped PCollection. -
  2. DatastoreWordCount - — An example that shows you how to read from Google Cloud Datastore.
  3. DistinctExample — An example that uses Shakespeare's plays as plain text files, and removes duplicate lines across all the files. Demonstrates the From 6489b6dbbcc947e707994892cbe1b6edb0e5b396 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 18 Nov 2016 09:54:36 -0800 Subject: [PATCH 073/279] Remove global default execution of exec-maven-plugin --- pom.xml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/pom.xml b/pom.xml index 4e60fabcdd6c..35a055cea55c 100644 --- a/pom.xml +++ b/pom.xml @@ -988,14 +988,6 @@ org.codehaus.mojo exec-maven-plugin 1.4.0 - - - verify - - java - - - false From 9cb8b5fa4c2275b2dda94483575a3aaf04dae34d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 10 Nov 2016 14:47:56 -0800 Subject: [PATCH 074/279] Create java8-examples archetype module This archetype module is automatically generated during the build process, and its dependencies and tests verified. --- .gitignore | 3 + .../examples-java8/generate-sources.sh | 82 +++++++ .../maven-archetypes/examples-java8/pom.xml | 177 +++++++++++++++ .../META-INF/maven/archetype-metadata.xml | 39 ++++ .../resources/archetype-resources/pom.xml | 209 ++++++++++++++++++ .../projects/basic/archetype.properties | 21 ++ .../test/resources/projects/basic/goal.txt | 1 + .../resources/archetype-resources/pom.xml | 19 +- sdks/java/maven-archetypes/pom.xml | 17 ++ 9 files changed, 567 insertions(+), 1 deletion(-) create mode 100755 sdks/java/maven-archetypes/examples-java8/generate-sources.sh create mode 100644 sdks/java/maven-archetypes/examples-java8/pom.xml create mode 100644 sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml create mode 100644 sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml create mode 100644 sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties create mode 100644 sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt diff --git a/.gitignore b/.gitignore index 0340d8a65742..fcfeafc0a91e 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,9 @@ target/ bin/ +# Ignore generated archetypes +sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/src/ + # Ignore IntelliJ files. .idea/ *.iml diff --git a/sdks/java/maven-archetypes/examples-java8/generate-sources.sh b/sdks/java/maven-archetypes/examples-java8/generate-sources.sh new file mode 100755 index 000000000000..7b85e854df37 --- /dev/null +++ b/sdks/java/maven-archetypes/examples-java8/generate-sources.sh @@ -0,0 +1,82 @@ +#!/bin/bash -ex +# +# 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. +# + +# Updates the examples-java8 archetype to have selection of our examples +# for use in walkthroughs, etc. +# +# Usage: Invoke with no arguments from any working directory. + +# The directory of this script. Assumes root of the maven-archetypes module. +HERE="$(dirname $0)" + +# The directory of the examples-java and examples-java8 modules +EXAMPLES_ROOT="${HERE}/../../../../examples/java" +JAVA8_EXAMPLES_ROOT="${HERE}/../../../../examples/java8" + +# The root of the examples archetype +ARCHETYPE_ROOT="${HERE}/src/main/resources/archetype-resources" + +mkdir -p "$ARCHETYPE_ROOT/src/main/java" +mkdir -p "$ARCHETYPE_ROOT/src/test/java" + +# +# Copy the Java 7 subset of the examples project verbatim. +# +rsync -a --exclude cookbook --exclude complete \ + "${EXAMPLES_ROOT}"/src/main/java/org/apache/beam/examples/ \ + "${ARCHETYPE_ROOT}/src/main/java" + +rsync -a --exclude cookbook --exclude complete --exclude '*IT.java' \ + "${EXAMPLES_ROOT}"/src/test/java/org/apache/beam/examples/ \ + "${ARCHETYPE_ROOT}/src/test/java" + +# +# Copy in MinimalWordCountJava8 and mobile gaming example +# +rsync -a \ + "${JAVA8_EXAMPLES_ROOT}"/src/main/java/org/apache/beam/examples/ \ + "${ARCHETYPE_ROOT}/src/main/java" + +rsync -a \ + "${JAVA8_EXAMPLES_ROOT}"/src/test/java/org/apache/beam/examples/ \ + "${ARCHETYPE_ROOT}/src/test/java" + + +# +# Replace 'package org.apache.beam.examples' with 'package ${package}' in all Java code +# +find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \ + | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g' + +find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \ + | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g' + +# +# Replace 'import org.apache.beam.examples.' with 'import ${package}.' in all Java code +# +find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \ + | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g' + +find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \ + | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g' + +# +# The use of -i.bak is necessary for the above to work with both GNU and BSD sed. +# Delete the files now. +# +find "${ARCHETYPE_ROOT}/src" -name '*.bak' -delete diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml new file mode 100644 index 000000000000..5cf094e36aee --- /dev/null +++ b/sdks/java/maven-archetypes/examples-java8/pom.xml @@ -0,0 +1,177 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-sdks-java-maven-archetypes-parent + 0.4.0-incubating-SNAPSHOT + ../pom.xml + + + beam-sdks-java-maven-archetypes-examples-java8 + Apache Beam :: SDKs :: Java :: Maven Archetypes :: Examples - Java 8 + A Maven Archetype to create a project containing + example pipelines from the Apache Beam Java SDK, targeting Java 8. + + + maven-archetype + + + + + org.apache.maven.archetype + archetype-packaging + 2.4 + + + + + + + maven-archetype-plugin + 2.4 + + + + default-integration-test + install + + integration-test + + + + + + + + + + exec-maven-plugin + org.codehaus.mojo + + + generate-archetype-contents + generate-sources + + exec + + + ${project.basedir}/generate-sources.sh + + + + + + + + + + + com.google.api-client + google-api-client + runtime + + + + com.google.apis + google-api-services-bigquery + runtime + + + + com.google.apis + google-api-services-pubsub + runtime + + + + com.google.guava + guava + runtime + + + + com.google.http-client + google-http-client + runtime + + + + com.google.oauth-client + google-oauth-client + runtime + + + + joda-time + joda-time + runtime + + + + org.apache.avro + avro + runtime + + + + org.apache.beam + beam-sdks-java-core + runtime + + + + org.apache.beam + beam-runners-direct-java + runtime + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + runtime + + + + org.apache.beam + beam-sdks-java-io-google-cloud-platform + runtime + + + + org.slf4j + slf4j-api + runtime + + + diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml new file mode 100644 index 000000000000..dbdd614719e6 --- /dev/null +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml @@ -0,0 +1,39 @@ + + + + + + + src/main/java + + **/*.java + + + + + src/test/java + + **/*.java + + + + diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml new file mode 100644 index 000000000000..ece84126fcf7 --- /dev/null +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -0,0 +1,209 @@ + + + + 4.0.0 + + ${groupId} + ${artifactId} + ${version} + + jar + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.5.1 + + 1.8 + 1.8 + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.19.1 + + all + 4 + true + + + + org.apache.maven.surefire + surefire-junit47 + 2.19.1 + + + + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.4.0 + + false + + + + + + + + + + org.apache.beam + beam-sdks-java-core + 0.4.0-incubating-SNAPSHOT + + + + + org.apache.beam + beam-runners-direct-java + 0.4.0-incubating-SNAPSHOT + runtime + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + 0.4.0-incubating-SNAPSHOT + runtime + + + + + org.apache.beam + beam-sdks-java-io-google-cloud-platform + 0.4.0-incubating-SNAPSHOT + + + + com.google.api-client + google-api-client + 1.22.0 + + + + com.google.guava + guava-jdk5 + + + + + + + com.google.apis + google-api-services-bigquery + v2-rev295-1.22.0 + + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client + 1.22.0 + + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-pubsub + v1-rev10-1.22.0 + + + + com.google.guava + guava-jdk5 + + + + + + joda-time + joda-time + 2.4 + + + + com.google.guava + guava + 19.0 + + + + + org.slf4j + slf4j-api + 1.7.14 + + + + org.slf4j + slf4j-jdk14 + 1.7.14 + + runtime + + + + + org.hamcrest + hamcrest-all + 1.3 + + + + junit + junit + 4.11 + + + + org.mockito + mockito-all + 1.9.5 + test + + + diff --git a/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties new file mode 100644 index 000000000000..ee81f8fbc5a0 --- /dev/null +++ b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties @@ -0,0 +1,21 @@ +# +# 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=it.pkg +version=0.1 +groupId=archetype.it +artifactId=basic +targetPlatform=1.8 diff --git a/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt new file mode 100644 index 000000000000..0b5987362fe3 --- /dev/null +++ b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt @@ -0,0 +1 @@ +verify diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index df2e9f351268..48f56fd7c3ee 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -85,7 +85,24 @@ false - + + + org.apache.maven.plugins + maven-dependency-plugin + 2.10 + + + analyze-only + + + true + true + + + + + + diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml index 02bb1507a2be..1b1c31905b81 100644 --- a/sdks/java/maven-archetypes/pom.xml +++ b/sdks/java/maven-archetypes/pom.xml @@ -31,6 +31,23 @@ Apache Beam :: SDKs :: Java :: Maven Archetypes + + + beam-sdks-java-archetypes-examples-java8 + + false + + unix + + [1.8,) + !beam.sdks.java.archetypes.examples.skip + + + examples-java8 + + + + starter examples From abd96b8986ef922057375279312fcd0fa66ab7ba Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 17 Nov 2016 15:19:16 -0800 Subject: [PATCH 075/279] Autogenerate examples archetype during build --- .gitignore | 1 + .../generate-sources.sh} | 21 +- sdks/java/maven-archetypes/examples/pom.xml | 27 +- .../resources/archetype-resources/pom.xml | 19 +- .../src/main/java/DebuggingWordCount.java | 164 -------- .../src/main/java/MinimalWordCount.java | 118 ------ .../src/main/java/WindowedWordCount.java | 229 ------------ .../src/main/java/WordCount.java | 186 --------- .../common/ExampleBigQueryTableOptions.java | 55 --- .../src/main/java/common/ExampleOptions.java | 37 -- ...mplePubsubTopicAndSubscriptionOptions.java | 45 --- .../common/ExamplePubsubTopicOptions.java | 45 --- .../src/main/java/common/ExampleUtils.java | 352 ------------------ .../src/test/java/DebuggingWordCountTest.java | 52 --- .../src/test/java/WordCountTest.java | 85 ----- sdks/java/maven-archetypes/pom.xml | 31 +- 16 files changed, 67 insertions(+), 1400 deletions(-) rename sdks/java/maven-archetypes/{update-examples-archetype.sh => examples/generate-sources.sh} (73%) delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java delete mode 100644 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java diff --git a/.gitignore b/.gitignore index fcfeafc0a91e..39e2f62cd892 100644 --- a/.gitignore +++ b/.gitignore @@ -7,6 +7,7 @@ target/ bin/ # Ignore generated archetypes +sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/ sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/src/ # Ignore IntelliJ files. diff --git a/sdks/java/maven-archetypes/update-examples-archetype.sh b/sdks/java/maven-archetypes/examples/generate-sources.sh similarity index 73% rename from sdks/java/maven-archetypes/update-examples-archetype.sh rename to sdks/java/maven-archetypes/examples/generate-sources.sh index 3c222be0d0f8..b1a86a7c06be 100755 --- a/sdks/java/maven-archetypes/update-examples-archetype.sh +++ b/sdks/java/maven-archetypes/examples/generate-sources.sh @@ -24,10 +24,13 @@ HERE="$(dirname $0)" # The directory of the examples-java module -EXAMPLES_ROOT="${HERE}/../../../examples/java" +EXAMPLES_ROOT="${HERE}/../../../../examples/java" # The root of the examples archetype -ARCHETYPE_ROOT="${HERE}/examples/src/main/resources/archetype-resources" +ARCHETYPE_ROOT="${HERE}/src/main/resources/archetype-resources" + +mkdir -p "${ARCHETYPE_ROOT}/src/main/java" +mkdir -p "${ARCHETYPE_ROOT}/src/test/java" # # Copy the Java subset of the examples project verbatim. @@ -44,16 +47,22 @@ rsync -a --exclude cookbook --exclude complete --exclude '*IT.java' \ # Replace 'package org.apache.beam.examples' with 'package ${package}' in all Java code # find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \ - | xargs -0 sed -i 's/^package org\.apache\.beam\.examples/package ${package}/g' + | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g' find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \ - | xargs -0 sed -i 's/^package org\.apache\.beam\.examples/package ${package}/g' + | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g' # # Replace 'import org.apache.beam.examples.' with 'import ${package}.' in all Java code # find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \ - | xargs -0 sed -i 's/^import org\.apache\.beam\.examples/import ${package}/g' + | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g' find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \ - | xargs -0 sed -i 's/^import org\.apache\.beam\.examples/import ${package}/g' + | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g' + +# +# The use of -i.bak is necessary for the above to work with both GNU and BSD sed. +# Delete the files now. +# +find "${ARCHETYPE_ROOT}/src" -name '*.bak' -delete diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml index b8555bca46be..637231fc7c9b 100644 --- a/sdks/java/maven-archetypes/examples/pom.xml +++ b/sdks/java/maven-archetypes/examples/pom.xml @@ -15,7 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -29,7 +31,8 @@ beam-sdks-java-maven-archetypes-examples Apache Beam :: SDKs :: Java :: Maven Archetypes :: Examples A Maven Archetype to create a project containing all the - example pipelines from the Apache Beam Java SDK. + example pipelines from the Apache Beam Java SDK. + maven-archetype @@ -65,7 +68,27 @@ + + + + exec-maven-plugin + org.codehaus.mojo + + + generate-archetype-contents + generate-sources + + exec + + + ${project.basedir}/generate-sources.sh + + + + + + org.apache.beam diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 48f56fd7c3ee..df2e9f351268 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -85,24 +85,7 @@ false - - - org.apache.maven.plugins - maven-dependency-plugin - 2.10 - - - analyze-only - - - true - true - - - - - - + diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java deleted file mode 100644 index 99ae79687cb0..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java +++ /dev/null @@ -1,164 +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 ${package}; - -import java.util.Arrays; -import java.util.List; -import java.util.regex.Pattern; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * An example that verifies word counts in Shakespeare and includes Beam best practices. - * - *

    This class, {@link DebuggingWordCount}, is the third in a series of four successively more - * detailed 'word count' examples. You may first want to take a look at {@link MinimalWordCount} - * and {@link WordCount}. After you've looked at this example, then see the - * {@link WindowedWordCount} pipeline, for introduction of additional concepts. - * - *

    Basic concepts, also in the MinimalWordCount and WordCount examples: - * Reading text files; counting a PCollection; executing a Pipeline both locally - * and using a selected runner; defining DoFns. - * - *

    New Concepts: - *

    - *   1. Logging using SLF4J, even in a distributed environment
    - *   2. Creating a custom aggregator (runners have varying levels of support)
    - *   3. Testing your Pipeline via PAssert
    - * 
    - * - *

    To execute this pipeline locally, specify general pipeline configuration: - *

    {@code
    - *   --project=YOUR_PROJECT_ID
    - * }
    - * 
    - * - *

    To change the runner, specify: - *

    {@code
    - *   --runner=YOUR_SELECTED_RUNNER
    - * }
    - * 
    - * - *

    The input file defaults to a public data set containing the text of of King Lear, - * by William Shakespeare. You can override it and choose your own input with {@code --inputFile}. - * - */ -public class DebuggingWordCount { - /** A DoFn that filters for a specific key based upon a regular expression. */ - public static class FilterTextFn extends DoFn, KV> { - /** - * Concept #1: The logger below uses the fully qualified class name of FilterTextFn as the - * logger. Depending on your SLF4J configuration, log statements will likely be qualified by - * this name. - * - *

    Note that this is entirely standard SLF4J usage. Some runners may provide a default SLF4J - * configuration that is most appropriate for their logging integration. - */ - private static final Logger LOG = LoggerFactory.getLogger(FilterTextFn.class); - - private final Pattern filter; - public FilterTextFn(String pattern) { - filter = Pattern.compile(pattern); - } - - /** - * Concept #2: A custom aggregator can track values in your pipeline as it runs. Each - * runner provides varying levels of support for aggregators, and may expose them - * in a dashboard, etc. - */ - private final Aggregator matchedWords = - createAggregator("matchedWords", new Sum.SumLongFn()); - private final Aggregator unmatchedWords = - createAggregator("umatchedWords", new Sum.SumLongFn()); - - @ProcessElement - public void processElement(ProcessContext c) { - if (filter.matcher(c.element().getKey()).matches()) { - // Log at the "DEBUG" level each element that we match. When executing this pipeline - // these log lines will appear only if the log level is set to "DEBUG" or lower. - LOG.debug("Matched: " + c.element().getKey()); - matchedWords.addValue(1L); - c.output(c.element()); - } else { - // Log at the "TRACE" level each element that is not matched. Different log levels - // can be used to control the verbosity of logging providing an effective mechanism - // to filter less important information. - LOG.trace("Did not match: " + c.element().getKey()); - unmatchedWords.addValue(1L); - } - } - } - - /** - * Options supported by {@link DebuggingWordCount}. - * - *

    Inherits standard configuration options and all options defined in - * {@link WordCount.WordCountOptions}. - */ - public interface WordCountOptions extends WordCount.WordCountOptions { - - @Description("Regex filter pattern to use in DebuggingWordCount. " - + "Only words matching this pattern will be counted.") - @Default.String("Flourish|stomach") - String getFilterPattern(); - void setFilterPattern(String value); - } - - public static void main(String[] args) { - WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() - .as(WordCountOptions.class); - Pipeline p = Pipeline.create(options); - - PCollection> filteredWords = - p.apply("ReadLines", TextIO.Read.from(options.getInputFile())) - .apply(new WordCount.CountWords()) - .apply(ParDo.of(new FilterTextFn(options.getFilterPattern()))); - - /** - * Concept #3: PAssert is a set of convenient PTransforms in the style of - * Hamcrest's collection matchers that can be used when writing Pipeline level tests - * to validate the contents of PCollections. PAssert is best used in unit tests - * with small data sets but is demonstrated here as a teaching tool. - * - *

    Below we verify that the set of filtered words matches our expected counts. Note - * that PAssert does not provide any output and that successful completion of the - * Pipeline implies that the expectations were met. Learn more at - * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline on how to test - * your Pipeline and see {@link DebuggingWordCountTest} for an example unit test. - */ - List> expectedResults = Arrays.asList( - KV.of("Flourish", 3L), - KV.of("stomach", 1L)); - PAssert.that(filteredWords).containsInAnyOrder(expectedResults); - - p.run().waitUntilFinish(); - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java deleted file mode 100644 index 97bd8243b87f..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java +++ /dev/null @@ -1,118 +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 ${package}; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.values.KV; - - -/** - * An example that counts words in Shakespeare. - * - *

    This class, {@link MinimalWordCount}, is the first in a series of four successively more - * detailed 'word count' examples. Here, for simplicity, we don't show any error-checking or - * argument processing, and focus on construction of the pipeline, which chains together the - * application of core transforms. - * - *

    Next, see the {@link WordCount} pipeline, then the {@link DebuggingWordCount}, and finally the - * {@link WindowedWordCount} pipeline, for more detailed examples that introduce additional - * concepts. - * - *

    Concepts: - * - *

    - *   1. Reading data from text files
    - *   2. Specifying 'inline' transforms
    - *   3. Counting items in a PCollection
    - *   4. Writing data to text files
    - * 
    - * - *

    No arguments are required to run this pipeline. It will be executed with the DirectRunner. You - * can see the results in the output files in your current working directory, with names like - * "wordcounts-00001-of-00005. When running on a distributed service, you would use an appropriate - * file service. - */ -public class MinimalWordCount { - - public static void main(String[] args) { - // Create a PipelineOptions object. This object lets us set various execution - // options for our pipeline, such as the runner you wish to use. This example - // will run with the DirectRunner by default, based on the class path configured - // in its dependencies. - PipelineOptions options = PipelineOptionsFactory.create(); - - // Create the Pipeline object with the options we defined above. - Pipeline p = Pipeline.create(options); - - // Apply the pipeline's transforms. - - // Concept #1: Apply a root transform to the pipeline; in this case, TextIO.Read to read a set - // of input text files. TextIO.Read returns a PCollection where each element is one line from - // the input text (a set of Shakespeare's texts). - - // This example reads a public data set consisting of the complete works of Shakespeare. - p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) - - // Concept #2: Apply a ParDo transform to our PCollection of text lines. This ParDo invokes a - // DoFn (defined in-line) on each element that tokenizes the text line into individual words. - // The ParDo returns a PCollection, where each element is an individual word in - // Shakespeare's collected texts. - .apply("ExtractWords", ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - for (String word : c.element().split("[^a-zA-Z']+")) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - })) - - // Concept #3: Apply the Count transform to our PCollection of individual words. The Count - // transform returns a new PCollection of key/value pairs, where each key represents a unique - // word in the text. The associated value is the occurrence count for that word. - .apply(Count.perElement()) - - // Apply a MapElements transform that formats our PCollection of word counts into a printable - // string, suitable for writing to an output file. - .apply("FormatResults", MapElements.via(new SimpleFunction, String>() { - @Override - public String apply(KV input) { - return input.getKey() + ": " + input.getValue(); - } - })) - - // Concept #4: Apply a write transform, TextIO.Write, at the end of the pipeline. - // TextIO.Write writes the contents of a PCollection (in this case, our PCollection of - // formatted strings) to a series of text files. - // - // By default, it will write to a set of files with names like wordcount-00001-of-00005 - .apply(TextIO.Write.to("wordcounts")); - - // Run the pipeline. - p.run().waitUntilFinish(); - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java deleted file mode 100644 index 28125319e212..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java +++ /dev/null @@ -1,229 +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 ${package}; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import ${package}.common.ExampleBigQueryTableOptions; -import ${package}.common.ExampleOptions; -import ${package}.common.ExampleUtils; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.joda.time.Duration; -import org.joda.time.Instant; - - -/** - * An example that counts words in text, and can run over either unbounded or bounded input - * collections. - * - *

    This class, {@link WindowedWordCount}, is the last in a series of four successively more - * detailed 'word count' examples. First take a look at {@link MinimalWordCount}, - * {@link WordCount}, and {@link DebuggingWordCount}. - * - *

    Basic concepts, also in the MinimalWordCount, WordCount, and DebuggingWordCount examples: - * Reading text files; counting a PCollection; writing to GCS; executing a Pipeline both locally - * and using a selected runner; defining DoFns; creating a custom aggregator; - * user-defined PTransforms; defining PipelineOptions. - * - *

    New Concepts: - *

    - *   1. Unbounded and bounded pipeline input modes
    - *   2. Adding timestamps to data
    - *   3. Windowing
    - *   4. Re-using PTransforms over windowed PCollections
    - *   5. Writing to BigQuery
    - * 
    - * - *

    By default, the examples will run with the {@code DirectRunner}. - * To change the runner, specify: - *

    {@code
    - *   --runner=YOUR_SELECTED_RUNNER
    - * }
    - * 
    - * See examples/java/README.md for instructions about how to configure different runners. - * - *

    Optionally specify the input file path via: - * {@code --inputFile=gs://INPUT_PATH}, - * which defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt}. - * - *

    Specify an output BigQuery dataset and optionally, a table for the output. If you don't - * specify the table, one will be created for you using the job name. If you don't specify the - * dataset, a dataset called {@code beam_examples} must already exist in your project. - * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}. - * - *

    By default, the pipeline will do fixed windowing, on 1-minute windows. You can - * change this interval by setting the {@code --windowSize} parameter, e.g. {@code --windowSize=10} - * for 10-minute windows. - * - *

    The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C) - * and then exits. - */ -public class WindowedWordCount { - static final int WINDOW_SIZE = 1; // Default window duration in minutes - - /** - * Concept #2: A DoFn that sets the data element timestamp. This is a silly method, just for - * this example, for the bounded data case. - * - *

    Imagine that many ghosts of Shakespeare are all typing madly at the same time to recreate - * his masterworks. Each line of the corpus will get a random associated timestamp somewhere in a - * 2-hour period. - */ - static class AddTimestampFn extends DoFn { - private static final Duration RAND_RANGE = Duration.standardHours(2); - private final Instant minTimestamp; - - AddTimestampFn() { - this.minTimestamp = new Instant(System.currentTimeMillis()); - } - - @ProcessElement - public void processElement(ProcessContext c) { - // Generate a timestamp that falls somewhere in the past two hours. - long randMillis = (long) (Math.random() * RAND_RANGE.getMillis()); - Instant randomTimestamp = minTimestamp.plus(randMillis); - /** - * Concept #2: Set the data element with that timestamp. - */ - c.outputWithTimestamp(c.element(), new Instant(randomTimestamp)); - } - } - - /** A DoFn that converts a Word and Count into a BigQuery table row. */ - static class FormatAsTableRowFn extends DoFn, TableRow> { - @ProcessElement - public void processElement(ProcessContext c) { - TableRow row = new TableRow() - .set("word", c.element().getKey()) - .set("count", c.element().getValue()) - // include a field for the window timestamp - .set("window_timestamp", c.timestamp().toString()); - c.output(row); - } - } - - /** - * Helper method that defines the BigQuery schema used for the output. - */ - private static TableSchema getSchema() { - List fields = new ArrayList<>(); - fields.add(new TableFieldSchema().setName("word").setType("STRING")); - fields.add(new TableFieldSchema().setName("count").setType("INTEGER")); - fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP")); - TableSchema schema = new TableSchema().setFields(fields); - return schema; - } - - /** - * Concept #5: We'll stream the results to a BigQuery table. The BigQuery output source is one - * that supports both bounded and unbounded data. This is a helper method that creates a - * TableReference from input options, to tell the pipeline where to write its BigQuery results. - */ - private static TableReference getTableReference(Options options) { - TableReference tableRef = new TableReference(); - tableRef.setProjectId(options.getProject()); - tableRef.setDatasetId(options.getBigQueryDataset()); - tableRef.setTableId(options.getBigQueryTable()); - return tableRef; - } - - /** - * Options supported by {@link WindowedWordCount}. - * - *

    Inherits standard example configuration options, which allow specification of the BigQuery - * table, as well as the {@link WordCount.WordCountOptions} support for - * specification of the input file. - */ - public interface Options extends WordCount.WordCountOptions, - ExampleOptions, ExampleBigQueryTableOptions { - @Description("Fixed window duration, in minutes") - @Default.Integer(WINDOW_SIZE) - Integer getWindowSize(); - void setWindowSize(Integer value); - } - - public static void main(String[] args) throws IOException { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - options.setBigQuerySchema(getSchema()); - // ExampleUtils creates the necessary input sources to simplify execution of this Pipeline. - ExampleUtils exampleUtils = new ExampleUtils(options); - exampleUtils.setup(); - - Pipeline pipeline = Pipeline.create(options); - - /** - * Concept #1: the Beam SDK lets us run the same pipeline with either a bounded or - * unbounded input source. - */ - PCollection input = pipeline - /** Read from the GCS file. */ - .apply(TextIO.Read.from(options.getInputFile())) - // Concept #2: Add an element timestamp, using an artificial time just to show windowing. - // See AddTimestampFn for more detail on this. - .apply(ParDo.of(new AddTimestampFn())); - - /** - * Concept #3: Window into fixed windows. The fixed window size for this example defaults to 1 - * minute (you can change this with a command-line option). See the documentation for more - * information on how fixed windows work, and for information on the other types of windowing - * available (e.g., sliding windows). - */ - PCollection windowedWords = input - .apply(Window.into( - FixedWindows.of(Duration.standardMinutes(options.getWindowSize())))); - - /** - * Concept #4: Re-use our existing CountWords transform that does not have knowledge of - * windows over a PCollection containing windowed values. - */ - PCollection> wordCounts = windowedWords.apply(new WordCount.CountWords()); - - /** - * Concept #5: Format the results for a BigQuery table, then write to BigQuery. - * The BigQuery output source supports both bounded and unbounded data. - */ - wordCounts.apply(ParDo.of(new FormatAsTableRowFn())) - .apply(BigQueryIO.Write - .to(getTableReference(options)) - .withSchema(getSchema()) - .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) - .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND)); - - PipelineResult result = pipeline.run(); - - // ExampleUtils will try to cancel the pipeline before the program exists. - exampleUtils.waitToFinish(result); - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java deleted file mode 100644 index 8fe71370c74f..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java +++ /dev/null @@ -1,186 +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 ${package}; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - -/** - * An example that counts words in Shakespeare and includes Beam best practices. - * - *

    This class, {@link WordCount}, is the second in a series of four successively more detailed - * 'word count' examples. You may first want to take a look at {@link MinimalWordCount}. - * After you've looked at this example, then see the {@link DebuggingWordCount} - * pipeline, for introduction of additional concepts. - * - *

    For a detailed walkthrough of this example, see - * - * http://beam.incubator.apache.org/use/walkthroughs/ - * - * - *

    Basic concepts, also in the MinimalWordCount example: - * Reading text files; counting a PCollection; writing to text files - * - *

    New Concepts: - *

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

    Concept #1: you can execute this pipeline either locally or using by selecting another runner. - * These are now command-line options and not hard-coded as they were in the MinimalWordCount - * example. - * - *

    To change the runner, specify: - *

    {@code
    - *   --runner=YOUR_SELECTED_RUNNER
    - * }
    - * 
    - * - *

    To execute this pipeline, specify a local output file (if using the - * {@code DirectRunner}) or output prefix on a supported distributed file system. - *

    {@code
    - *   --output=[YOUR_LOCAL_FILE | YOUR_OUTPUT_PREFIX]
    - * }
    - * - *

    The input file defaults to a public data set containing the text of of King Lear, - * by William Shakespeare. You can override it and choose your own input with {@code --inputFile}. - */ -public class WordCount { - - /** - * Concept #2: You can make your pipeline assembly code less verbose by defining your DoFns - * statically out-of-line. This DoFn tokenizes lines of text into individual words; we pass it - * to a ParDo in the pipeline. - */ - static class ExtractWordsFn extends DoFn { - private final Aggregator emptyLines = - createAggregator("emptyLines", new Sum.SumLongFn()); - - @ProcessElement - public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.addValue(1L); - } - - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - /** A SimpleFunction that converts a Word and Count into a printable string. */ - public static class FormatAsTextFn extends SimpleFunction, String> { - @Override - public String apply(KV input) { - return input.getKey() + ": " + input.getValue(); - } - } - - /** - * A PTransform that converts a PCollection containing lines of text into a PCollection of - * formatted word counts. - * - *

    Concept #3: This is a custom composite transform that bundles two transforms (ParDo and - * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse, - * modular testing, and an improved monitoring experience. - */ - public static class CountWords extends PTransform, - PCollection>> { - @Override - public PCollection> apply(PCollection lines) { - - // Convert lines of text into individual words. - PCollection words = lines.apply( - ParDo.of(new ExtractWordsFn())); - - // Count the number of times each word occurs. - PCollection> wordCounts = - words.apply(Count.perElement()); - - return wordCounts; - } - } - - /** - * Options supported by {@link WordCount}. - * - *

    Concept #4: Defining your own configuration options. Here, you can add your own arguments - * to be processed by the command-line parser, and specify default values for them. You can then - * access the options values in your pipeline code. - * - *

    Inherits standard configuration options. - */ - public interface WordCountOptions extends PipelineOptions { - - /** - * By default, this example reads from a public dataset containing the text of - * King Lear. Set this option to choose a different input file or glob. - */ - @Description("Path of the file to read from") - @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") - String getInputFile(); - void setInputFile(String value); - - /** - * Set this required option to specify where to write the output. - */ - @Description("Path of the file to write to") - @Required - String getOutput(); - void setOutput(String value); - } - - public static void main(String[] args) { - WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() - .as(WordCountOptions.class); - Pipeline p = Pipeline.create(options); - - // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the - // static FormatAsTextFn() to the ParDo transform. - p.apply("ReadLines", TextIO.Read.from(options.getInputFile())) - .apply(new CountWords()) - .apply(MapElements.via(new FormatAsTextFn())) - .apply("WriteCounts", TextIO.Write.to(options.getOutput())); - - p.run().waitUntilFinish(); - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java deleted file mode 100644 index 6b51074f44d0..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java +++ /dev/null @@ -1,55 +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 ${package}.common; - -import com.google.api.services.bigquery.model.TableSchema; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * Options that can be used to configure BigQuery tables in Beam examples. - * The project defaults to the project being used to run the example. - */ -public interface ExampleBigQueryTableOptions extends GcpOptions { - @Description("BigQuery dataset name") - @Default.String("beam_examples") - String getBigQueryDataset(); - void setBigQueryDataset(String dataset); - - @Description("BigQuery table name") - @Default.InstanceFactory(BigQueryTableFactory.class) - String getBigQueryTable(); - void setBigQueryTable(String table); - - @Description("BigQuery table schema") - TableSchema getBigQuerySchema(); - void setBigQuerySchema(TableSchema schema); - - /** - * Returns the job name as the default BigQuery table name. - */ - class BigQueryTableFactory implements DefaultValueFactory { - @Override - public String create(PipelineOptions options) { - return options.getJobName().replace('-', '_'); - } - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java deleted file mode 100644 index 90f935c3cec0..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java +++ /dev/null @@ -1,37 +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 ${package}.common; - -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * Options that can be used to configure the Beam examples. - */ -public interface ExampleOptions extends PipelineOptions { - @Description("Whether to keep jobs running after local process exit") - @Default.Boolean(false) - boolean getKeepJobsRunning(); - void setKeepJobsRunning(boolean keepJobsRunning); - - @Description("Number of workers to use when executing the injector pipeline") - @Default.Integer(1) - int getInjectorNumWorkers(); - void setInjectorNumWorkers(int numWorkers); -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java deleted file mode 100644 index daeb398f7fca..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java +++ /dev/null @@ -1,45 +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 ${package}.common; - -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * Options that can be used to configure Pub/Sub topic/subscription in Beam examples. - */ -public interface ExamplePubsubTopicAndSubscriptionOptions extends ExamplePubsubTopicOptions { - @Description("Pub/Sub subscription") - @Default.InstanceFactory(PubsubSubscriptionFactory.class) - String getPubsubSubscription(); - void setPubsubSubscription(String subscription); - - /** - * Returns a default Pub/Sub subscription based on the project and the job names. - */ - class PubsubSubscriptionFactory implements DefaultValueFactory { - @Override - public String create(PipelineOptions options) { - return "projects/" + options.as(GcpOptions.class).getProject() - + "/subscriptions/" + options.getJobName(); - } - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java deleted file mode 100644 index 936bff567507..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java +++ /dev/null @@ -1,45 +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 ${package}.common; - -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * Options that can be used to configure Pub/Sub topic in Beam examples. - */ -public interface ExamplePubsubTopicOptions extends GcpOptions { - @Description("Pub/Sub topic") - @Default.InstanceFactory(PubsubTopicFactory.class) - String getPubsubTopic(); - void setPubsubTopic(String topic); - - /** - * Returns a default Pub/Sub topic based on the project and the job names. - */ - class PubsubTopicFactory implements DefaultValueFactory { - @Override - public String create(PipelineOptions options) { - return "projects/" + options.as(GcpOptions.class).getProject() - + "/topics/" + options.getJobName(); - } - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java deleted file mode 100644 index 570b3827b784..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java +++ /dev/null @@ -1,352 +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 ${package}.common; - -import com.google.api.client.googleapis.json.GoogleJsonResponseException; -import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; -import com.google.api.services.bigquery.Bigquery; -import com.google.api.services.bigquery.Bigquery.Datasets; -import com.google.api.services.bigquery.Bigquery.Tables; -import com.google.api.services.bigquery.model.Dataset; -import com.google.api.services.bigquery.model.DatasetReference; -import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.Subscription; -import com.google.api.services.pubsub.model.Topic; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.util.concurrent.Uninterruptibles; -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PubsubOptions; -import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.Transport; -import org.joda.time.Duration; - -/** - * The utility class that sets up and tears down external resources, - * and cancels the streaming pipelines once the program terminates. - * - *

    It is used to run Beam examples. - */ -public class ExampleUtils { - - private static final int SC_NOT_FOUND = 404; - - private final PipelineOptions options; - private Bigquery bigQueryClient = null; - private Pubsub pubsubClient = null; - private Set pipelinesToCancel = Sets.newHashSet(); - private List pendingMessages = Lists.newArrayList(); - - /** - * Do resources and runner options setup. - */ - public ExampleUtils(PipelineOptions options) { - this.options = options; - } - - /** - * Sets up external resources that are required by the example, - * such as Pub/Sub topics and BigQuery tables. - * - * @throws IOException if there is a problem setting up the resources - */ - public void setup() throws IOException { - Sleeper sleeper = Sleeper.DEFAULT; - BackOff backOff = - FluentBackoff.DEFAULT - .withMaxRetries(3).withInitialBackoff(Duration.millis(200)).backoff(); - Throwable lastException = null; - try { - do { - try { - setupPubsub(); - setupBigQueryTable(); - return; - } catch (GoogleJsonResponseException e) { - lastException = e; - } - } while (BackOffUtils.next(sleeper, backOff)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - // Ignore InterruptedException - } - throw new RuntimeException(lastException); - } - - /** - * Sets up the Google Cloud Pub/Sub topic. - * - *

    If the topic doesn't exist, a new topic with the given name will be created. - * - * @throws IOException if there is a problem setting up the Pub/Sub topic - */ - public void setupPubsub() throws IOException { - ExamplePubsubTopicAndSubscriptionOptions pubsubOptions = - options.as(ExamplePubsubTopicAndSubscriptionOptions.class); - if (!pubsubOptions.getPubsubTopic().isEmpty()) { - pendingMessages.add("**********************Set Up Pubsub************************"); - setupPubsubTopic(pubsubOptions.getPubsubTopic()); - pendingMessages.add("The Pub/Sub topic has been set up for this example: " - + pubsubOptions.getPubsubTopic()); - - if (!pubsubOptions.getPubsubSubscription().isEmpty()) { - setupPubsubSubscription( - pubsubOptions.getPubsubTopic(), pubsubOptions.getPubsubSubscription()); - pendingMessages.add("The Pub/Sub subscription has been set up for this example: " - + pubsubOptions.getPubsubSubscription()); - } - } - } - - /** - * Sets up the BigQuery table with the given schema. - * - *

    If the table already exists, the schema has to match the given one. Otherwise, the example - * will throw a RuntimeException. If the table doesn't exist, a new table with the given schema - * will be created. - * - * @throws IOException if there is a problem setting up the BigQuery table - */ - public void setupBigQueryTable() throws IOException { - ExampleBigQueryTableOptions bigQueryTableOptions = - options.as(ExampleBigQueryTableOptions.class); - if (bigQueryTableOptions.getBigQueryDataset() != null - && bigQueryTableOptions.getBigQueryTable() != null - && bigQueryTableOptions.getBigQuerySchema() != null) { - pendingMessages.add("******************Set Up Big Query Table*******************"); - setupBigQueryTable(bigQueryTableOptions.getProject(), - bigQueryTableOptions.getBigQueryDataset(), - bigQueryTableOptions.getBigQueryTable(), - bigQueryTableOptions.getBigQuerySchema()); - pendingMessages.add("The BigQuery table has been set up for this example: " - + bigQueryTableOptions.getProject() - + ":" + bigQueryTableOptions.getBigQueryDataset() - + "." + bigQueryTableOptions.getBigQueryTable()); - } - } - - /** - * Tears down external resources that can be deleted upon the example's completion. - */ - private void tearDown() { - pendingMessages.add("*************************Tear Down*************************"); - ExamplePubsubTopicAndSubscriptionOptions pubsubOptions = - options.as(ExamplePubsubTopicAndSubscriptionOptions.class); - if (!pubsubOptions.getPubsubTopic().isEmpty()) { - try { - deletePubsubTopic(pubsubOptions.getPubsubTopic()); - pendingMessages.add("The Pub/Sub topic has been deleted: " - + pubsubOptions.getPubsubTopic()); - } catch (IOException e) { - pendingMessages.add("Failed to delete the Pub/Sub topic : " - + pubsubOptions.getPubsubTopic()); - } - if (!pubsubOptions.getPubsubSubscription().isEmpty()) { - try { - deletePubsubSubscription(pubsubOptions.getPubsubSubscription()); - pendingMessages.add("The Pub/Sub subscription has been deleted: " - + pubsubOptions.getPubsubSubscription()); - } catch (IOException e) { - pendingMessages.add("Failed to delete the Pub/Sub subscription : " - + pubsubOptions.getPubsubSubscription()); - } - } - } - - ExampleBigQueryTableOptions bigQueryTableOptions = - options.as(ExampleBigQueryTableOptions.class); - if (bigQueryTableOptions.getBigQueryDataset() != null - && bigQueryTableOptions.getBigQueryTable() != null - && bigQueryTableOptions.getBigQuerySchema() != null) { - pendingMessages.add("The BigQuery table might contain the example's output, " - + "and it is not deleted automatically: " - + bigQueryTableOptions.getProject() - + ":" + bigQueryTableOptions.getBigQueryDataset() - + "." + bigQueryTableOptions.getBigQueryTable()); - pendingMessages.add("Please go to the Developers Console to delete it manually." - + " Otherwise, you may be charged for its usage."); - } - } - - private void setupBigQueryTable(String projectId, String datasetId, String tableId, - TableSchema schema) throws IOException { - if (bigQueryClient == null) { - bigQueryClient = Transport.newBigQueryClient(options.as(BigQueryOptions.class)).build(); - } - - Datasets datasetService = bigQueryClient.datasets(); - if (executeNullIfNotFound(datasetService.get(projectId, datasetId)) == null) { - Dataset newDataset = new Dataset().setDatasetReference( - new DatasetReference().setProjectId(projectId).setDatasetId(datasetId)); - datasetService.insert(projectId, newDataset).execute(); - } - - Tables tableService = bigQueryClient.tables(); - Table table = executeNullIfNotFound(tableService.get(projectId, datasetId, tableId)); - if (table == null) { - Table newTable = new Table().setSchema(schema).setTableReference( - new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId)); - tableService.insert(projectId, datasetId, newTable).execute(); - } else if (!table.getSchema().equals(schema)) { - throw new RuntimeException( - "Table exists and schemas do not match, expecting: " + schema.toPrettyString() - + ", actual: " + table.getSchema().toPrettyString()); - } - } - - private void setupPubsubTopic(String topic) throws IOException { - if (pubsubClient == null) { - pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build(); - } - if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) == null) { - pubsubClient.projects().topics().create(topic, new Topic().setName(topic)).execute(); - } - } - - private void setupPubsubSubscription(String topic, String subscription) throws IOException { - if (pubsubClient == null) { - pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build(); - } - if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) == null) { - Subscription subInfo = new Subscription() - .setAckDeadlineSeconds(60) - .setTopic(topic); - pubsubClient.projects().subscriptions().create(subscription, subInfo).execute(); - } - } - - /** - * Deletes the Google Cloud Pub/Sub topic. - * - * @throws IOException if there is a problem deleting the Pub/Sub topic - */ - private void deletePubsubTopic(String topic) throws IOException { - if (pubsubClient == null) { - pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build(); - } - if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) != null) { - pubsubClient.projects().topics().delete(topic).execute(); - } - } - - /** - * Deletes the Google Cloud Pub/Sub subscription. - * - * @throws IOException if there is a problem deleting the Pub/Sub subscription - */ - private void deletePubsubSubscription(String subscription) throws IOException { - if (pubsubClient == null) { - pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build(); - } - if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) != null) { - pubsubClient.projects().subscriptions().delete(subscription).execute(); - } - } - - /** - * Waits for the pipeline to finish and cancels it before the program exists. - */ - public void waitToFinish(PipelineResult result) { - pipelinesToCancel.add(result); - if (!options.as(ExampleOptions.class).getKeepJobsRunning()) { - addShutdownHook(pipelinesToCancel); - } - try { - result.waitUntilFinish(); - } catch (UnsupportedOperationException e) { - // Do nothing if the given PipelineResult doesn't support waitUntilFinish(), - // such as EvaluationResults returned by DirectRunner. - tearDown(); - printPendingMessages(); - } catch (Exception e) { - throw new RuntimeException("Failed to wait the pipeline until finish: " + result); - } - } - - private void addShutdownHook(final Collection pipelineResults) { - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run() { - tearDown(); - printPendingMessages(); - for (PipelineResult pipelineResult : pipelineResults) { - try { - pipelineResult.cancel(); - } catch (IOException e) { - System.out.println("Failed to cancel the job."); - System.out.println(e.getMessage()); - } - } - - for (PipelineResult pipelineResult : pipelineResults) { - boolean cancellationVerified = false; - for (int retryAttempts = 6; retryAttempts > 0; retryAttempts--) { - if (pipelineResult.getState().isTerminal()) { - cancellationVerified = true; - break; - } else { - System.out.println( - "The example pipeline is still running. Verifying the cancellation."); - } - Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); - } - if (!cancellationVerified) { - System.out.println("Failed to verify the cancellation for job: " + pipelineResult); - } - } - } - }); - } - - private void printPendingMessages() { - System.out.println(); - System.out.println("***********************************************************"); - System.out.println("***********************************************************"); - for (String message : pendingMessages) { - System.out.println(message); - } - System.out.println("***********************************************************"); - System.out.println("***********************************************************"); - } - - private static T executeNullIfNotFound( - AbstractGoogleClientRequest request) throws IOException { - try { - return request.execute(); - } catch (GoogleJsonResponseException e) { - if (e.getStatusCode() == SC_NOT_FOUND) { - return null; - } else { - throw e; - } - } - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java deleted file mode 100644 index 155242d9969f..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java +++ /dev/null @@ -1,52 +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 ${package}; - -import com.google.common.io.Files; -import java.io.File; -import java.nio.charset.StandardCharsets; -import ${package}.DebuggingWordCount.WordCountOptions; -import org.apache.beam.sdk.testing.TestPipeline; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests for {@link DebuggingWordCount}. - */ -@RunWith(JUnit4.class) -public class DebuggingWordCountTest { - @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - - @Test - public void testDebuggingWordCount() throws Exception { - File inputFile = tmpFolder.newFile(); - File outputFile = tmpFolder.newFile(); - Files.write( - "stomach secret Flourish message Flourish here Flourish", - inputFile, - StandardCharsets.UTF_8); - WordCountOptions options = - TestPipeline.testingPipelineOptions().as(WordCountOptions.class); - options.setInputFile(inputFile.getAbsolutePath()); - options.setOutput(outputFile.getAbsolutePath()); - DebuggingWordCount.main(TestPipeline.convertToArgs(options)); - } -} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java deleted file mode 100644 index e86c2aac969c..000000000000 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java +++ /dev/null @@ -1,85 +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 ${package}; - -import java.util.Arrays; -import java.util.List; -import ${package}.WordCount.CountWords; -import ${package}.WordCount.ExtractWordsFn; -import ${package}.WordCount.FormatAsTextFn; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -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.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnTester; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.values.PCollection; -import org.hamcrest.CoreMatchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests of WordCount. - */ -@RunWith(JUnit4.class) -public class WordCountTest { - - /** Example test that tests a specific {@link DoFn}. */ - @Test - public void testExtractWordsFn() throws Exception { - DoFnTester extractWordsFn = - DoFnTester.of(new ExtractWordsFn()); - - Assert.assertThat(extractWordsFn.processBundle(" some input words "), - CoreMatchers.hasItems("some", "input", "words")); - Assert.assertThat(extractWordsFn.processBundle(" "), - CoreMatchers.hasItems()); - Assert.assertThat(extractWordsFn.processBundle(" some ", " input", " words"), - CoreMatchers.hasItems("some", "input", "words")); - } - - static final String[] WORDS_ARRAY = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; - - static final List WORDS = Arrays.asList(WORDS_ARRAY); - - static final String[] COUNTS_ARRAY = new String[] { - "hi: 5", "there: 1", "sue: 2", "bob: 2"}; - - /** Example test that tests a PTransform by using an in-memory input and inspecting the output. */ - @Test - @Category(RunnableOnService.class) - public void testCountWords() throws Exception { - Pipeline p = TestPipeline.create(); - - PCollection input = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of())); - - PCollection output = input.apply(new CountWords()) - .apply(MapElements.via(new FormatAsTextFn())); - - PAssert.that(output).containsInAnyOrder(COUNTS_ARRAY); - p.run().waitUntilFinish(); - } -} diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml index 1b1c31905b81..be9ad6a752ac 100644 --- a/sdks/java/maven-archetypes/pom.xml +++ b/sdks/java/maven-archetypes/pom.xml @@ -31,7 +31,32 @@ Apache Beam :: SDKs :: Java :: Maven Archetypes + + starter + + + + + beam-sdks-java-archetypes-examples + + false + + unix + + !beam.sdks.java.archetypes.examples.skip + + + examples + + + beam-sdks-java-archetypes-examples-java8 @@ -47,10 +72,4 @@ - - - starter - examples - - From 58916b94ada142d63d16a315da58f88184995f10 Mon Sep 17 00:00:00 2001 From: Daniel Kulp Date: Mon, 28 Nov 2016 16:47:37 -0500 Subject: [PATCH 076/279] [BEAM-1057] Fix JDBC test derby startup issues --- .../apache/beam/sdk/io/jdbc/JdbcIOTest.java | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java index fe574af9dc73..d09929ddd17c 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java @@ -20,7 +20,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.PrintWriter; import java.io.Serializable; +import java.io.StringWriter; import java.net.InetAddress; import java.net.ServerSocket; import java.sql.Connection; @@ -73,7 +75,29 @@ public static void startDatabase() throws Exception { System.setProperty("derby.stream.error.file", "target/derby.log"); derbyServer = new NetworkServerControl(InetAddress.getByName("localhost"), port); - derbyServer.start(null); + StringWriter out = new StringWriter(); + derbyServer.start(new PrintWriter(out)); + boolean started = false; + int count = 0; + // Use two different methods to detect when server is started: + // 1) Check the server stdout for the "started" string + // 2) wait up to 15 seconds for the derby server to start based on a ping + // on faster machines and networks, this may return very quick, but on slower + // networks where the DNS lookups are slow, this may take a little time + while (!started && count < 30) { + if (out.toString().contains("started")) { + started = true; + } else { + count++; + Thread.sleep(500); + try { + derbyServer.ping(); + started = true; + } catch (Throwable t) { + //ignore, still trying to start + } + } + } dataSource = new ClientDataSource(); dataSource.setCreateDatabase("create"); From b1a67934d1496e221718599301635c38f8e3b7ec Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Mon, 28 Nov 2016 11:11:10 +0200 Subject: [PATCH 077/279] [BEAM-1000, BEAM-1050] Fixed PipelineResult.State Failed for streaming, support non-blocking cancel/waituntilfinish in batch. Added a SparkPipelineResult class to address PipelineResult#waitUntilFinish() and SparkRunner#run() semantics. * Simplified beamExceptionFrom() to abstract away SparkExceptions. * Reordered methods according to access level. --- .../beam/runners/spark/EvaluationResult.java | 67 ------ .../runners/spark/SparkPipelineResult.java | 179 ++++++++++++++ .../beam/runners/spark/SparkRunner.java | 98 ++++---- .../beam/runners/spark/TestSparkRunner.java | 10 +- .../runners/spark/examples/WordCount.java | 2 +- .../spark/translation/EvaluationContext.java | 119 ++-------- .../translation/SparkContextFactory.java | 2 +- .../spark/ProvidedSparkContextTest.java | 6 +- .../runners/spark/SparkPipelineStateTest.java | 219 ++++++++++++++++++ .../metrics/sink/ClearAggregatorsRule.java | 2 +- .../metrics/sink/NamedAggregatorsTest.java | 2 +- .../runners/spark/io/AvroPipelineTest.java | 2 +- .../beam/runners/spark/io/NumShardsTest.java | 2 +- .../hadoop/HadoopFileFormatPipelineTest.java | 2 +- .../spark/translation/SideEffectsTest.java | 59 ----- .../streaming/EmptyStreamAssertionTest.java | 4 + .../ResumeFromCheckpointStreamingTest.java | 8 +- .../streaming/utils/PAssertStreaming.java | 9 +- 18 files changed, 500 insertions(+), 292 deletions(-) delete mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java delete mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java deleted file mode 100644 index 52606a35927c..000000000000 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/EvaluationResult.java +++ /dev/null @@ -1,67 +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.spark; - -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PValue; - -/** - * Interface for retrieving the result(s) of running a pipeline. Allows us to translate between - * {@code PObject}s or {@code PCollection}s and Ts or collections of Ts. - */ -public interface EvaluationResult extends PipelineResult { - /** - * Retrieves an iterable of results associated with the PCollection passed in. - * - * @param pcollection Collection we wish to translate. - * @param Type of elements contained in collection. - * @return Natively types result associated with collection. - */ - Iterable get(PCollection pcollection); - - /** - * Retrieve an object of Type T associated with the PValue passed in. - * - * @param pval PValue to retrieve associated data for. - * @param Type of object to return. - * @return Native object. - */ - T get(PValue pval); - - /** - * Retrieves the final value of the aggregator. - * - * @param aggName name of aggregator. - * @param resultType Class of final result of aggregation. - * @param Type of final result of aggregation. - * @return Result of aggregation associated with specified name. - */ - T getAggregatorValue(String aggName, Class resultType); - - /** - * Releases any runtime resources, including distributed-execution contexts currently held by - * this EvaluationResult; once close() has been called, - * {@link EvaluationResult#get(PCollection)} might - * not work for subsequent calls. - * - * @param gracefully true if Spark should finish all ongoing work before closing. - */ - void close(boolean gracefully); -} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java new file mode 100644 index 000000000000..ec0610cdb99d --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark; + +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.beam.runners.spark.translation.EvaluationContext; +import org.apache.beam.runners.spark.translation.SparkContextFactory; +import org.apache.beam.sdk.AggregatorRetrievalException; +import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.spark.SparkException; +import org.joda.time.Duration; + +/** + * Represents a Spark pipeline execution result. + */ +public abstract class SparkPipelineResult implements PipelineResult { + + protected final Future pipelineExecution; + protected final EvaluationContext context; + + protected PipelineResult.State state; + + SparkPipelineResult(final Future pipelineExecution, + final EvaluationContext evaluationContext) { + this.pipelineExecution = pipelineExecution; + this.context = evaluationContext; + // pipelineExecution is expected to have started executing eagerly. + state = State.RUNNING; + } + + private RuntimeException runtimeExceptionFrom(Throwable e) { + return (e instanceof RuntimeException) ? (RuntimeException) e : new RuntimeException(e); + } + + private RuntimeException beamExceptionFrom(Throwable e) { + // Scala doesn't declare checked exceptions in the bytecode, and the Java compiler + // won't let you catch something that is not declared, so we can't catch + // SparkException directly, instead we do an instanceof check. + return (e instanceof SparkException) + ? new Pipeline.PipelineExecutionException(e.getCause() != null ? e.getCause() : e) + : runtimeExceptionFrom(e); + } + + protected abstract void stop(); + + protected abstract State awaitTermination(Duration duration) + throws TimeoutException, ExecutionException, InterruptedException; + + public T getAggregatorValue(String named, Class resultType) { + return context.getAggregatorValue(named, resultType); + } + + @Override + public PipelineResult.State getState() { + return state; + } + + @Override + public PipelineResult.State waitUntilFinish() { + return waitUntilFinish(Duration.millis(Long.MAX_VALUE)); + } + + @Override + public State waitUntilFinish(Duration duration) { + try { + state = awaitTermination(duration); + } catch (TimeoutException e) { + state = null; + } catch (ExecutionException e) { + state = PipelineResult.State.FAILED; + throw beamExceptionFrom(e.getCause()); + } catch (Exception e) { + state = PipelineResult.State.FAILED; + throw beamExceptionFrom(e); + } finally { + stop(); + } + + return state; + } + + @Override + public AggregatorValues getAggregatorValues(Aggregator aggregator) + throws AggregatorRetrievalException { + return context.getAggregatorValues(aggregator); + } + + @Override + public MetricResults metrics() { + throw new UnsupportedOperationException("The SparkRunner does not currently support metrics."); + } + + @Override + public PipelineResult.State cancel() throws IOException { + if (state != null && !state.isTerminal()) { + stop(); + state = PipelineResult.State.CANCELLED; + } + + return state; + } + + /** + * Represents the result of running a batch pipeline. + */ + static class BatchMode extends SparkPipelineResult { + + BatchMode(final Future pipelineExecution, + final EvaluationContext evaluationContext) { + super(pipelineExecution, evaluationContext); + } + + @Override + protected void stop() { + SparkContextFactory.stopSparkContext(context.getSparkContext()); + } + + @Override + protected State awaitTermination(Duration duration) + throws TimeoutException, ExecutionException, InterruptedException { + pipelineExecution.get(duration.getMillis(), TimeUnit.MILLISECONDS); + return PipelineResult.State.DONE; + } + } + + /** + * Represents a streaming Spark pipeline result. + */ + static class StreamingMode extends SparkPipelineResult { + + StreamingMode(final Future pipelineExecution, + final EvaluationContext evaluationContext) { + super(pipelineExecution, evaluationContext); + } + + @Override + protected void stop() { + context.getStreamingContext().stop(false, true); + SparkContextFactory.stopSparkContext(context.getSparkContext()); + } + + @Override + protected State awaitTermination(Duration duration) throws TimeoutException, + ExecutionException, InterruptedException { + pipelineExecution.get(duration.getMillis(), TimeUnit.MILLISECONDS); + if (context.getStreamingContext().awaitTerminationOrTimeout(duration.getMillis())) { + return State.DONE; + } else { + return null; + } + } + + } + +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 63f77c0b14a7..a8c600efda1a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -20,6 +20,9 @@ import java.util.Collection; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.runners.spark.translation.SparkPipelineTranslator; @@ -36,14 +39,12 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.PBegin; 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.spark.SparkException; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.slf4j.Logger; @@ -58,7 +59,7 @@ * * {@code * Pipeline p = [logic for pipeline creation] - * EvaluationResult result = (EvaluationResult) p.run(); + * SparkPipelineResult result = (SparkPipelineResult) p.run(); * } * *

    To create a pipeline runner to run against a different spark cluster, with a custom master url @@ -68,10 +69,10 @@ * Pipeline p = [logic for pipeline creation] * SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); * options.setSparkMaster("spark://host:port"); - * EvaluationResult result = (EvaluationResult) p.run(); + * SparkPipelineResult result = (SparkPipelineResult) p.run(); * } */ -public final class SparkRunner extends PipelineRunner { +public final class SparkRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(SparkRunner.class); /** @@ -122,50 +123,57 @@ private SparkRunner(SparkPipelineOptions options) { } @Override - public EvaluationResult run(Pipeline pipeline) { - try { - LOG.info("Executing pipeline using the SparkRunner."); - - detectTranslationMode(pipeline); - if (mOptions.isStreaming()) { - SparkRunnerStreamingContextFactory contextFactory = - new SparkRunnerStreamingContextFactory(pipeline, mOptions); - JavaStreamingContext jssc = JavaStreamingContext.getOrCreate(mOptions.getCheckpointDir(), - contextFactory); - - LOG.info("Starting streaming pipeline execution."); - jssc.start(); - - // if recovering from checkpoint, we have to reconstruct the EvaluationResult instance. - return contextFactory.getCtxt() == null ? new EvaluationContext(jssc.sparkContext(), - pipeline, jssc) : contextFactory.getCtxt(); - } else { - JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); - SparkPipelineTranslator translator = new TransformTranslator.Translator(); - pipeline.traverseTopologically(new Evaluator(translator, ctxt)); - ctxt.computeOutputs(); + public SparkPipelineResult run(final Pipeline pipeline) { + LOG.info("Executing pipeline using the SparkRunner."); + + final SparkPipelineResult result; + final EvaluationContext evaluationContext; + final Future startPipeline; + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + + detectTranslationMode(pipeline); + + if (mOptions.isStreaming()) { + final SparkRunnerStreamingContextFactory contextFactory = + new SparkRunnerStreamingContextFactory(pipeline, mOptions); + final JavaStreamingContext jssc = + JavaStreamingContext.getOrCreate(mOptions.getCheckpointDir(), contextFactory); + + // if recovering from checkpoint, we have to reconstruct the Evaluation instance. + evaluationContext = + contextFactory.getCtxt() == null + ? new EvaluationContext(jssc.sparkContext(), pipeline, jssc) + : contextFactory.getCtxt(); + + startPipeline = executorService.submit(new Runnable() { + + @Override + public void run() { + LOG.info("Starting streaming pipeline execution."); + jssc.start(); + } + }); - LOG.info("Pipeline execution complete."); + result = new SparkPipelineResult.StreamingMode(startPipeline, evaluationContext); + } else { + final JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions); + evaluationContext = new EvaluationContext(jsc, pipeline); - return ctxt; - } - } catch (Exception e) { - // Scala doesn't declare checked exceptions in the bytecode, and the Java compiler - // won't let you catch something that is not declared, so we can't catch - // SparkException here. Instead we do an instanceof check. - // Then we find the cause by seeing if it's a user exception (wrapped by Beam's - // UserCodeException), or just use the SparkException cause. - if (e instanceof SparkException && e.getCause() != null) { - if (e.getCause() instanceof UserCodeException && e.getCause().getCause() != null) { - throw UserCodeException.wrap(e.getCause().getCause()); - } else { - throw new RuntimeException(e.getCause()); + startPipeline = executorService.submit(new Runnable() { + + @Override + public void run() { + pipeline.traverseTopologically(new Evaluator(new TransformTranslator.Translator(), + evaluationContext)); + evaluationContext.computeOutputs(); + LOG.info("Batch pipeline execution complete."); } - } - // otherwise just wrap in a RuntimeException - throw new RuntimeException(e); + }); + + result = new SparkPipelineResult.BatchMode(startPipeline, evaluationContext); } + + return result; } /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index a4ddca042bf6..9a67f9c84405 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -37,7 +37,7 @@ * * {@code * Pipeline p = [logic for pipeline creation] - * EvaluationResult result = (EvaluationResult) p.run(); + * SparkPipelineResult result = (SparkPipelineResult) p.run(); * } * *

    To create a pipeline runner to run against a different spark cluster, with a custom master url @@ -47,10 +47,10 @@ * Pipeline p = [logic for pipeline creation] * SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); * options.setSparkMaster("spark://host:port"); - * EvaluationResult result = (EvaluationResult) p.run(); + * SparkPipelineResult result = (SparkPipelineResult) p.run(); * } */ -public final class TestSparkRunner extends PipelineRunner { +public final class TestSparkRunner extends PipelineRunner { private SparkRunner delegate; @@ -72,9 +72,9 @@ OutputT apply(PTransform transform, InputT input) { }; @Override - public EvaluationResult run(Pipeline pipeline) { + public SparkPipelineResult run(Pipeline pipeline) { TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class); - EvaluationResult result = delegate.run(pipeline); + SparkPipelineResult result = delegate.run(pipeline); assertThat(result, testPipelineOptions.getOnCreateMatcher()); assertThat(result, testPipelineOptions.getOnSuccessMatcher()); return result; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java index 0ae78f233185..38dae38b92be 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java @@ -132,6 +132,6 @@ public static void main(String[] args) { .apply(MapElements.via(new FormatAsTextFn())) .apply("WriteCounts", TextIO.Write.to(options.getOutput())); - p.run(); + p.run().waitUntilFinish(); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index ae456098dc22..425f1141ec4e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -21,12 +21,10 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.Iterables; -import java.io.IOException; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.translation.streaming.UnboundedDataset; @@ -34,7 +32,6 @@ import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -46,15 +43,13 @@ import org.apache.beam.sdk.values.PValue; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.streaming.StreamingContextState; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.joda.time.Duration; - /** - * Evaluation context allows us to define how pipeline instructions. + * The EvaluationContext allows us to define pipeline instructions and translate between + * {@code PObject}s or {@code PCollection}s and Ts or DStreams/RDDs of Ts. */ -public class EvaluationContext implements EvaluationResult { +public class EvaluationContext { private final JavaSparkContext jsc; private JavaStreamingContext jssc; private final SparkRuntimeContext runtime; @@ -66,24 +61,20 @@ public class EvaluationContext implements EvaluationResult { private final Map pobjects = new LinkedHashMap<>(); private final Map>> pview = new LinkedHashMap<>(); private AppliedPTransform currentTransform; - private State state; public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; this.pipeline = pipeline; this.runtime = new SparkRuntimeContext(pipeline, jsc); - // A batch pipeline is blocking by nature - this.state = State.DONE; } public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, JavaStreamingContext jssc) { this(jsc, pipeline); this.jssc = jssc; - this.state = State.RUNNING; } - JavaSparkContext getSparkContext() { + public JavaSparkContext getSparkContext() { return jsc; } @@ -179,8 +170,14 @@ public void computeOutputs() { } } + /** + * Retrieve an object of Type T associated with the PValue passed in. + * + * @param value PValue to retrieve associated data for. + * @param Type of object to return. + * @return Native object. + */ @SuppressWarnings("unchecked") - @Override public T get(PValue value) { if (pobjects.containsKey(value)) { T result = (T) pobjects.get(value); @@ -195,23 +192,24 @@ public T get(PValue value) { throw new IllegalStateException("Cannot resolve un-known PObject: " + value); } - @Override - public T getAggregatorValue(String named, Class resultType) { - return runtime.getAggregatorValue(AccumulatorSingleton.getInstance(jsc), named, resultType); - } - - @Override public AggregatorValues getAggregatorValues(Aggregator aggregator) throws AggregatorRetrievalException { return runtime.getAggregatorValues(AccumulatorSingleton.getInstance(jsc), aggregator); } - @Override - public MetricResults metrics() { - throw new UnsupportedOperationException("The SparkRunner does not currently support metrics."); + public T getAggregatorValue(String named, Class resultType) { + return runtime.getAggregatorValue(AccumulatorSingleton.getInstance(jsc), + named, + resultType); } - @Override + /** + * Retrieves an iterable of results associated with the PCollection passed in. + * + * @param pcollection Collection we wish to translate. + * @param Type of elements contained in collection. + * @return Natively types result associated with collection. + */ public Iterable get(PCollection pcollection) { @SuppressWarnings("unchecked") BoundedDataset boundedDataset = (BoundedDataset) datasets.get(pcollection); @@ -225,79 +223,6 @@ Iterable> getWindowedValues(PCollection pcollection) { return boundedDataset.getValues(pcollection); } - @Override - public void close(boolean gracefully) { - // Stopping streaming job if running - if (isStreamingPipeline() && !state.isTerminal()) { - try { - cancel(gracefully); - } catch (IOException e) { - throw new RuntimeException("Failed to cancel streaming job", e); - } - } - SparkContextFactory.stopSparkContext(jsc); - } - - @Override - public State getState() { - return state; - } - - @Override - public State cancel() throws IOException { - return cancel(true); - } - - private State cancel(boolean gracefully) throws IOException { - if (isStreamingPipeline()) { - if (!state.isTerminal()) { - jssc.stop(false, gracefully); - state = State.CANCELLED; - } - return state; - } else { - // Batch is currently blocking so - // there is no way to cancel a batch job - // will be handled at BEAM-1000 - throw new UnsupportedOperationException( - "Spark runner EvaluationContext does not support cancel."); - } - } - - @Override - public State waitUntilFinish() { - return waitUntilFinish(Duration.ZERO); - } - - @Override - public State waitUntilFinish(Duration duration) { - if (isStreamingPipeline()) { - // According to PipelineResult: Provide a value less than 1 ms for an infinite wait - if (duration.getMillis() < 1L) { - jssc.awaitTermination(); - state = State.DONE; - } else { - jssc.awaitTermination(duration.getMillis()); - // According to PipelineResult: The final state of the pipeline or null on timeout - if (jssc.getState().equals(StreamingContextState.STOPPED)) { - state = State.DONE; - } else { - return null; - } - } - return state; - } else { - // This is no-op, since Spark runner in batch is blocking. - // It needs to be updated once SparkRunner supports non-blocking execution: - // https://issues.apache.org/jira/browse/BEAM-595 - return State.DONE; - } - } - - private boolean isStreamingPipeline() { - return jssc != null; - } - private String storageLevel() { return runtime.getPipelineOptions().as(SparkPipelineOptions.class).getStorageLevel(); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java index c7f90b40023f..67839a808243 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java @@ -66,7 +66,7 @@ public static synchronized JavaSparkContext getSparkContext(SparkPipelineOptions } } - static synchronized void stopSparkContext(JavaSparkContext context) { + public static synchronized void stopSparkContext(JavaSparkContext context) { if (!Boolean.getBoolean(TEST_REUSE_SPARK_CONTEXT)) { context.stop(); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java index fe73abaf7b41..298284453c39 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java @@ -76,7 +76,7 @@ public void testWithProvidedContext() throws Exception { PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); // Run test from pipeline - p.run(); + p.run().waitUntilFinish(); jsc.stop(); } @@ -100,7 +100,7 @@ public void testWithNullContext() throws Exception { PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); try { - p.run(); + p.run().waitUntilFinish(); fail("Should throw an exception when The provided Spark context is null"); } catch (RuntimeException e){ assert(e.getMessage().contains(PROVIDED_CONTEXT_EXCEPTION)); @@ -128,7 +128,7 @@ public void testWithStoppedProvidedContext() throws Exception { PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); try { - p.run(); + p.run().waitUntilFinish(); fail("Should throw an exception when The provided Spark context is stopped"); } catch (RuntimeException e){ assert(e.getMessage().contains(PROVIDED_CONTEXT_EXCEPTION)); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java new file mode 100644 index 000000000000..69cf1c4b37dc --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.runners.spark.io.CreateStream; +import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptions; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +/** + * This suite tests that various scenarios result in proper states of the pipeline. + */ +public class SparkPipelineStateTest implements Serializable { + + private static class UserException extends RuntimeException { + + UserException(String message) { + super(message); + } + } + + @Rule + public transient SparkTestPipelineOptions commonOptions = new SparkTestPipelineOptions(); + + @Rule + public transient TestName testName = new TestName(); + + private static final String FAILED_THE_BATCH_INTENTIONALLY = "Failed the batch intentionally"; + + private static final List BATCH_WORDS = Arrays.asList("one", "two"); + + private static final List> STREAMING_WORDS = + Lists.>newArrayList(BATCH_WORDS); + + private ParDo.Bound printParDo(final String prefix) { + return ParDo.of(new DoFn() { + + @ProcessElement + public void processElement(ProcessContext c) { + System.out.println(prefix + " " + c.element()); + } + }); + } + + private PTransform> getValues(SparkPipelineOptions options) { + return options.isStreaming() + ? CreateStream.fromQueue(STREAMING_WORDS) + : Create.of(BATCH_WORDS); + } + + private SparkPipelineOptions getStreamingOptions() { + final SparkPipelineOptions options = commonOptions.getOptions(); + options.setStreaming(true); + return options; + } + + private SparkPipelineOptions getBatchOptions() { + return commonOptions.getOptions(); + } + + private Pipeline getPipeline(SparkPipelineOptions options) { + + final Pipeline pipeline = Pipeline.create(options); + final String name = testName.getMethodName() + "(isStreaming=" + options.isStreaming() + ")"; + + pipeline + .apply(getValues(options)).setCoder(StringUtf8Coder.of()) + .apply(printParDo(name)); + + return pipeline; + } + + private void testFailedPipeline(SparkPipelineOptions options) throws Exception { + + SparkPipelineResult result = null; + + try { + final Pipeline pipeline = Pipeline.create(options); + pipeline + .apply(getValues(options)).setCoder(StringUtf8Coder.of()) + .apply(MapElements.via(new SimpleFunction() { + + @Override + public String apply(String input) { + throw new UserException(FAILED_THE_BATCH_INTENTIONALLY); + } + })); + + result = (SparkPipelineResult) pipeline.run(); + result.waitUntilFinish(); + } catch (Exception e) { + assertThat(e, instanceOf(Pipeline.PipelineExecutionException.class)); + assertThat(e.getCause(), instanceOf(UserCodeException.class)); + assertThat(e.getCause().getCause(), instanceOf(UserException.class)); + assertThat(e.getCause().getCause().getMessage(), is(FAILED_THE_BATCH_INTENTIONALLY)); + assertThat(result.getState(), is(PipelineResult.State.FAILED)); + result.cancel(); + return; + } + + fail("An injected failure did not affect the pipeline as expected."); + } + + private void testTimeoutPipeline(SparkPipelineOptions options) throws Exception { + + final Pipeline pipeline = getPipeline(options); + + SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); + + result.waitUntilFinish(Duration.millis(1)); + + assertThat(result.getState(), nullValue()); + + result.cancel(); + } + + private void testCanceledPipeline(SparkPipelineOptions options) throws Exception { + + final Pipeline pipeline = getPipeline(options); + + SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); + + result.cancel(); + + assertThat(result.getState(), is(PipelineResult.State.CANCELLED)); + } + + private void testRunningPipeline(SparkPipelineOptions options) throws Exception { + + final Pipeline pipeline = getPipeline(options); + + SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); + + assertThat(result.getState(), is(PipelineResult.State.RUNNING)); + + result.cancel(); + } + + @Test + public void testStreamingPipelineRunningState() throws Exception { + testRunningPipeline(getStreamingOptions()); + } + + @Test + public void testBatchPipelineRunningState() throws Exception { + testRunningPipeline(getBatchOptions()); + } + + @Test + public void testStreamingPipelineCanceledState() throws Exception { + testCanceledPipeline(getStreamingOptions()); + } + + @Test + public void testBatchPipelineCanceledState() throws Exception { + testCanceledPipeline(getBatchOptions()); + } + + @Test + public void testStreamingPipelineFailedState() throws Exception { + testFailedPipeline(getStreamingOptions()); + } + + @Test + public void testBatchPipelineFailedState() throws Exception { + testFailedPipeline(getBatchOptions()); + } + + @Test + public void testStreamingPipelineTimeoutState() throws Exception { + testTimeoutPipeline(getStreamingOptions()); + } + + @Test + public void testBatchPipelineTimeoutState() throws Exception { + testTimeoutPipeline(getBatchOptions()); + } + +} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java index 79c58a797465..52ae01918328 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java @@ -25,7 +25,7 @@ * A rule that clears the {@link org.apache.beam.runners.spark.aggregators.AccumulatorSingleton} * which represents the Beam {@link org.apache.beam.sdk.transforms.Aggregator}s. */ -class ClearAggregatorsRule extends ExternalResource { +public class ClearAggregatorsRule extends ExternalResource { @Override protected void before() throws Throwable { AccumulatorSingleton.clear(); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java index c16574c188bd..6b36bcc23a13 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java @@ -78,7 +78,7 @@ private void runPipeline() { PAssert.that(output).containsInAnyOrder(expectedCounts); - pipeline.run(); + pipeline.run().waitUntilFinish(); } @Test diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java index 03f96d502841..c5bb583abc40 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java @@ -76,7 +76,7 @@ public void testGeneric() throws Exception { PCollection input = p.apply( AvroIO.Read.from(inputFile.getAbsolutePath()).withSchema(schema)); input.apply(AvroIO.Write.to(outputDir.getAbsolutePath()).withSchema(schema)); - p.run(); + p.run().waitUntilFinish(); List records = readGenericFile(); assertEquals(Lists.newArrayList(savedRecord), records); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java index 4e5435fd3d7c..34d68189f6fc 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java @@ -74,7 +74,7 @@ public void testText() throws Exception { PCollection output = inputWords.apply(new WordCount.CountWords()) .apply(MapElements.via(new WordCount.FormatAsTextFn())); output.apply(TextIO.Write.to(outputDir.getAbsolutePath()).withNumShards(3).withSuffix(".txt")); - p.run(); + p.run().waitUntilFinish(); int count = 0; Set expected = Sets.newHashSet("hi: 5", "there: 1", "sue: 2", "bob: 2"); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java index b68e8d49e6f3..9efc670cf4fa 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/HadoopFileFormatPipelineTest.java @@ -88,7 +88,7 @@ public void testSequenceFile() throws Exception { HadoopIO.Write.Bound write = HadoopIO.Write.to(outputFile.getAbsolutePath(), outputFormatClass, IntWritable.class, Text.class); input.apply(write.withoutSharding()); - p.run(); + p.run().waitUntilFinish(); IntWritable key = new IntWritable(); Text value = new Text(); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java deleted file mode 100644 index 3b79d036c217..000000000000 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SideEffectsTest.java +++ /dev/null @@ -1,59 +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.spark.translation; - -import static org.hamcrest.core.Is.isA; - -import java.io.Serializable; -import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptions; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -/** - * Side effects test. - */ -public class SideEffectsTest implements Serializable { - private static class UserException extends RuntimeException { - } - - @Rule - public final transient SparkTestPipelineOptions pipelineOptions = new SparkTestPipelineOptions(); - @Rule - public final transient ExpectedException expectedException = ExpectedException.none(); - - @Test - public void test() throws Exception { - Pipeline p = Pipeline.create(pipelineOptions.getOptions()); - - p.apply(Create.of("a")).apply(ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - throw new UserException(); - } - })); - - expectedException.expectCause(isA(UserException.class)); - p.run(); - } -} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java index 656107a7ea27..e3561d60e145 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java @@ -23,6 +23,7 @@ import java.io.Serializable; import java.util.Collections; import org.apache.beam.runners.spark.SparkPipelineOptions; +import org.apache.beam.runners.spark.aggregators.metrics.sink.ClearAggregatorsRule; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming; import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptionsForStreaming; @@ -54,6 +55,9 @@ public class EmptyStreamAssertionTest implements Serializable { public SparkTestPipelineOptionsForStreaming commonOptions = new SparkTestPipelineOptionsForStreaming(); + @Rule + public ClearAggregatorsRule clearAggregatorsRule = new ClearAggregatorsRule(); + @Test public void testAssertion() throws Exception { SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java index b57787f76ba1..e0d71d4c94c4 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java @@ -27,8 +27,8 @@ import java.util.Map; import java.util.Properties; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; +import org.apache.beam.runners.spark.SparkPipelineResult; import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.translation.streaming.utils.EmbeddedKafkaCluster; import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming; @@ -118,7 +118,7 @@ public void testRun() throws Exception { options.setCheckpointDurationMillis(options.getBatchIntervalMillis()); // first run will read from Kafka backlog - "auto.offset.reset=smallest" - EvaluationResult res = run(options); + SparkPipelineResult res = run(options); long processedMessages1 = res.getAggregatorValue("processedMessages", Long.class); assertThat(String.format("Expected %d processed messages count but " + "found %d", EXPECTED_AGG_FIRST, processedMessages1), processedMessages1, @@ -132,14 +132,14 @@ public void testRun() throws Exception { equalTo(EXPECTED_AGG_FIRST)); } - private static EvaluationResult runAgain(SparkPipelineOptions options) { + private static SparkPipelineResult runAgain(SparkPipelineOptions options) { AccumulatorSingleton.clear(); // sleep before next run. Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); return run(options); } - private static EvaluationResult run(SparkPipelineOptions options) { + private static SparkPipelineResult run(SparkPipelineOptions options) { // write to Kafka produce(); Map consumerProps = ImmutableMap.of( diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java index 23aca4311392..471ec926ac51 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/PAssertStreaming.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertThat; import java.io.Serializable; -import org.apache.beam.runners.spark.EvaluationResult; +import org.apache.beam.runners.spark.SparkPipelineResult; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.Aggregator; @@ -55,7 +55,7 @@ private PAssertStreaming() { * Note that it is oblivious to windowing, so the assertion will apply indiscriminately to all * windows. */ - public static EvaluationResult runAndAssertContents(Pipeline p, + public static SparkPipelineResult runAndAssertContents(Pipeline p, PCollection actual, T[] expected, Duration timeout, @@ -69,9 +69,8 @@ public static EvaluationResult runAndAssertContents(Pipeline p, .apply(ParDo.of(new AssertDoFn<>(expected))); // run the pipeline. - EvaluationResult res = (EvaluationResult) p.run(); + SparkPipelineResult res = (SparkPipelineResult) p.run(); res.waitUntilFinish(timeout); - res.close(stopGracefully); // validate assertion succeeded (at least once). int success = res.getAggregatorValue(PAssert.SUCCESS_COUNTER, Integer.class); Assert.assertThat("Success aggregator should be greater than zero.", success, not(0)); @@ -87,7 +86,7 @@ public static EvaluationResult runAndAssertContents(Pipeline p, * Default to stop gracefully so that tests will finish processing even if slower for reasons * such as a slow runtime environment. */ - public static EvaluationResult runAndAssertContents(Pipeline p, + public static SparkPipelineResult runAndAssertContents(Pipeline p, PCollection actual, T[] expected, Duration timeout) { From 158378f0f682b80462b917002b895ddbf782d06d Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Sat, 3 Dec 2016 00:47:39 +0200 Subject: [PATCH 078/279] Redistributed some responsibilities in order to remove getAggregatorValues() form EvaluationContext. Inferred excepted exception handling according to existing codebase and tests. --- .../runners/spark/SparkPipelineResult.java | 76 +++++++++------ .../beam/runners/spark/SparkRunner.java | 35 +++++-- .../beam/runners/spark/TestSparkRunner.java | 1 + .../aggregators/AccumulatorSingleton.java | 6 +- .../spark/aggregators/SparkAggregators.java | 97 +++++++++++++++++++ .../spark/translation/EvaluationContext.java | 20 +--- .../translation/SparkRuntimeContext.java | 62 +----------- .../translation/TransformTranslator.java | 10 +- .../StreamingTransformTranslator.java | 10 +- .../runners/spark/SparkPipelineStateTest.java | 36 ++++--- .../sink => }/ClearAggregatorsRule.java | 8 +- .../metrics/sink/NamedAggregatorsTest.java | 1 + .../streaming/EmptyStreamAssertionTest.java | 2 +- .../ResumeFromCheckpointStreamingTest.java | 9 +- 14 files changed, 216 insertions(+), 157 deletions(-) create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java rename runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/{metrics/sink => }/ClearAggregatorsRule.java (89%) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java index ec0610cdb99d..b1027a6e96f7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java @@ -23,7 +23,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import org.apache.beam.runners.spark.translation.EvaluationContext; +import org.apache.beam.runners.spark.aggregators.SparkAggregators; import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; @@ -31,7 +31,10 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.util.UserCodeException; import org.apache.spark.SparkException; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.joda.time.Duration; /** @@ -40,29 +43,37 @@ public abstract class SparkPipelineResult implements PipelineResult { protected final Future pipelineExecution; - protected final EvaluationContext context; + protected JavaSparkContext javaSparkContext; protected PipelineResult.State state; SparkPipelineResult(final Future pipelineExecution, - final EvaluationContext evaluationContext) { + final JavaSparkContext javaSparkContext) { this.pipelineExecution = pipelineExecution; - this.context = evaluationContext; + this.javaSparkContext = javaSparkContext; // pipelineExecution is expected to have started executing eagerly. state = State.RUNNING; } - private RuntimeException runtimeExceptionFrom(Throwable e) { + private RuntimeException runtimeExceptionFrom(final Throwable e) { return (e instanceof RuntimeException) ? (RuntimeException) e : new RuntimeException(e); } - private RuntimeException beamExceptionFrom(Throwable e) { + private RuntimeException beamExceptionFrom(final Throwable e) { // Scala doesn't declare checked exceptions in the bytecode, and the Java compiler // won't let you catch something that is not declared, so we can't catch // SparkException directly, instead we do an instanceof check. - return (e instanceof SparkException) - ? new Pipeline.PipelineExecutionException(e.getCause() != null ? e.getCause() : e) - : runtimeExceptionFrom(e); + + if (e instanceof SparkException) { + if (e.getCause() != null && e.getCause() instanceof UserCodeException) { + UserCodeException userException = (UserCodeException) e.getCause(); + return new Pipeline.PipelineExecutionException(userException.getCause()); + } else if (e.getCause() != null) { + return new Pipeline.PipelineExecutionException(e.getCause()); + } + } + + return runtimeExceptionFrom(e); } protected abstract void stop(); @@ -70,8 +81,14 @@ private RuntimeException beamExceptionFrom(Throwable e) { protected abstract State awaitTermination(Duration duration) throws TimeoutException, ExecutionException, InterruptedException; - public T getAggregatorValue(String named, Class resultType) { - return context.getAggregatorValue(named, resultType); + public T getAggregatorValue(final String name, final Class resultType) { + return SparkAggregators.valueOf(name, resultType, javaSparkContext); + } + + @Override + public AggregatorValues getAggregatorValues(final Aggregator aggregator) + throws AggregatorRetrievalException { + return SparkAggregators.valueOf(aggregator, javaSparkContext); } @Override @@ -85,15 +102,15 @@ public PipelineResult.State waitUntilFinish() { } @Override - public State waitUntilFinish(Duration duration) { + public State waitUntilFinish(final Duration duration) { try { state = awaitTermination(duration); - } catch (TimeoutException e) { + } catch (final TimeoutException e) { state = null; - } catch (ExecutionException e) { + } catch (final ExecutionException e) { state = PipelineResult.State.FAILED; throw beamExceptionFrom(e.getCause()); - } catch (Exception e) { + } catch (final Exception e) { state = PipelineResult.State.FAILED; throw beamExceptionFrom(e); } finally { @@ -103,12 +120,6 @@ public State waitUntilFinish(Duration duration) { return state; } - @Override - public AggregatorValues getAggregatorValues(Aggregator aggregator) - throws AggregatorRetrievalException { - return context.getAggregatorValues(aggregator); - } - @Override public MetricResults metrics() { throw new UnsupportedOperationException("The SparkRunner does not currently support metrics."); @@ -130,17 +141,17 @@ public PipelineResult.State cancel() throws IOException { static class BatchMode extends SparkPipelineResult { BatchMode(final Future pipelineExecution, - final EvaluationContext evaluationContext) { - super(pipelineExecution, evaluationContext); + final JavaSparkContext javaSparkContext) { + super(pipelineExecution, javaSparkContext); } @Override protected void stop() { - SparkContextFactory.stopSparkContext(context.getSparkContext()); + SparkContextFactory.stopSparkContext(javaSparkContext); } @Override - protected State awaitTermination(Duration duration) + protected State awaitTermination(final Duration duration) throws TimeoutException, ExecutionException, InterruptedException { pipelineExecution.get(duration.getMillis(), TimeUnit.MILLISECONDS); return PipelineResult.State.DONE; @@ -152,22 +163,25 @@ protected State awaitTermination(Duration duration) */ static class StreamingMode extends SparkPipelineResult { + private final JavaStreamingContext javaStreamingContext; + StreamingMode(final Future pipelineExecution, - final EvaluationContext evaluationContext) { - super(pipelineExecution, evaluationContext); + final JavaStreamingContext javaStreamingContext) { + super(pipelineExecution, javaStreamingContext.sparkContext()); + this.javaStreamingContext = javaStreamingContext; } @Override protected void stop() { - context.getStreamingContext().stop(false, true); - SparkContextFactory.stopSparkContext(context.getSparkContext()); + javaStreamingContext.stop(false, true); + SparkContextFactory.stopSparkContext(javaSparkContext); } @Override - protected State awaitTermination(Duration duration) throws TimeoutException, + protected State awaitTermination(final Duration duration) throws TimeoutException, ExecutionException, InterruptedException { pipelineExecution.get(duration.getMillis(), TimeUnit.MILLISECONDS); - if (context.getStreamingContext().awaitTerminationOrTimeout(duration.getMillis())) { + if (javaStreamingContext.awaitTerminationOrTimeout(duration.getMillis())) { return State.DONE; } else { return null; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index a8c600efda1a..d51ee7d01960 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -23,6 +23,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.beam.runners.spark.aggregators.SparkAggregators; +import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetricSource; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.runners.spark.translation.SparkPipelineTranslator; @@ -45,7 +48,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; +import org.apache.spark.Accumulator; +import org.apache.spark.SparkEnv$; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.metrics.MetricsSystem; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,12 +128,25 @@ private SparkRunner(SparkPipelineOptions options) { mOptions = options; } + private void registerMetrics(final SparkPipelineOptions opts, final JavaSparkContext jsc) { + final Accumulator accum = SparkAggregators.getNamedAggregators(jsc); + final NamedAggregators initialValue = accum.value(); + + if (opts.getEnableSparkMetricSinks()) { + final MetricsSystem metricsSystem = SparkEnv$.MODULE$.get().metricsSystem(); + final AggregatorMetricSource aggregatorMetricSource = + new AggregatorMetricSource(opts.getAppName(), initialValue); + // re-register the metrics in case of context re-use + metricsSystem.removeSource(aggregatorMetricSource); + metricsSystem.registerSource(aggregatorMetricSource); + } + } + @Override public SparkPipelineResult run(final Pipeline pipeline) { LOG.info("Executing pipeline using the SparkRunner."); final SparkPipelineResult result; - final EvaluationContext evaluationContext; final Future startPipeline; final ExecutorService executorService = Executors.newSingleThreadExecutor(); @@ -139,30 +158,26 @@ public SparkPipelineResult run(final Pipeline pipeline) { final JavaStreamingContext jssc = JavaStreamingContext.getOrCreate(mOptions.getCheckpointDir(), contextFactory); - // if recovering from checkpoint, we have to reconstruct the Evaluation instance. - evaluationContext = - contextFactory.getCtxt() == null - ? new EvaluationContext(jssc.sparkContext(), pipeline, jssc) - : contextFactory.getCtxt(); - startPipeline = executorService.submit(new Runnable() { @Override public void run() { + registerMetrics(mOptions, jssc.sparkContext()); LOG.info("Starting streaming pipeline execution."); jssc.start(); } }); - result = new SparkPipelineResult.StreamingMode(startPipeline, evaluationContext); + result = new SparkPipelineResult.StreamingMode(startPipeline, jssc); } else { final JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions); - evaluationContext = new EvaluationContext(jsc, pipeline); + final EvaluationContext evaluationContext = new EvaluationContext(jsc, pipeline); startPipeline = executorService.submit(new Runnable() { @Override public void run() { + registerMetrics(mOptions, jsc); pipeline.traverseTopologically(new Evaluator(new TransformTranslator.Translator(), evaluationContext)); evaluationContext.computeOutputs(); @@ -170,7 +185,7 @@ public void run() { } }); - result = new SparkPipelineResult.BatchMode(startPipeline, evaluationContext); + result = new SparkPipelineResult.BatchMode(startPipeline, jsc); } return result; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 9a67f9c84405..2c26d84d4150 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -75,6 +75,7 @@ OutputT apply(PTransform transform, InputT input) { public SparkPipelineResult run(Pipeline pipeline) { TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class); SparkPipelineResult result = delegate.run(pipeline); + result.waitUntilFinish(); assertThat(result, testPipelineOptions.getOnCreateMatcher()); assertThat(result, testPipelineOptions.getOnSuccessMatcher()); return result; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java index bc7105fc93ff..883830e5ce8e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AccumulatorSingleton.java @@ -26,11 +26,11 @@ * For resilience, {@link Accumulator}s are required to be wrapped in a Singleton. * @see accumulators */ -public class AccumulatorSingleton { +class AccumulatorSingleton { private static volatile Accumulator instance = null; - public static Accumulator getInstance(JavaSparkContext jsc) { + static Accumulator getInstance(JavaSparkContext jsc) { if (instance == null) { synchronized (AccumulatorSingleton.class) { if (instance == null) { @@ -45,7 +45,7 @@ public static Accumulator getInstance(JavaSparkContext jsc) { } @VisibleForTesting - public static void clear() { + static void clear() { synchronized (AccumulatorSingleton.class) { instance = null; } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java new file mode 100644 index 000000000000..1b0669186f9e --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.aggregators; + +import com.google.common.collect.ImmutableList; +import java.util.Collection; +import java.util.Map; +import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.spark.Accumulator; +import org.apache.spark.api.java.JavaSparkContext; + +/** + * A utility class for retrieving aggregator values. + */ +public class SparkAggregators { + + private static AggregatorValues valueOf(final Accumulator accum, + final Aggregator aggregator) { + @SuppressWarnings("unchecked") + Class valueType = (Class) aggregator.getCombineFn().getOutputType().getRawType(); + final T value = valueOf(accum, aggregator.getName(), valueType); + + return new AggregatorValues() { + + @Override + public Collection getValues() { + return ImmutableList.of(value); + } + + @Override + public Map getValuesAtSteps() { + throw new UnsupportedOperationException("getValuesAtSteps is not supported."); + } + }; + } + + private static T valueOf(final Accumulator accum, + final String aggregatorName, + final Class typeClass) { + return accum.value().getValue(aggregatorName, typeClass); + } + + /** + * Retrieves the {@link NamedAggregators} instance using the provided Spark context. + * + * @param jsc a Spark context to be used in order to retrieve the name + * {@link NamedAggregators} instance + * @return a {@link NamedAggregators} instance + */ + public static Accumulator getNamedAggregators(JavaSparkContext jsc) { + return AccumulatorSingleton.getInstance(jsc); + } + + /** + * Retrieves the value of an aggregator from a SparkContext instance. + * + * @param aggregator The aggregator whose value to retrieve + * @param javaSparkContext The SparkContext instance + * @param The type of the aggregator's output + * @return The value of the aggregator + */ + public static AggregatorValues valueOf(final Aggregator aggregator, + final JavaSparkContext javaSparkContext) { + return valueOf(getNamedAggregators(javaSparkContext), aggregator); + } + + /** + * Retrieves the value of an aggregator from a SparkContext instance. + * + * @param name Name of the aggregator to retrieve the value of. + * @param typeClass Type class of value to be retrieved. + * @param Type of object to be returned. + * @return The value of the aggregator. + */ + public static T valueOf(final String name, + final Class typeClass, + final JavaSparkContext javaSparkContext) { + return valueOf(getNamedAggregators(javaSparkContext), name, typeClass); + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 425f1141ec4e..a412e313b857 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -26,13 +26,9 @@ import java.util.Map; import java.util.Set; import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.translation.streaming.UnboundedDataset; -import org.apache.beam.sdk.AggregatorRetrievalException; -import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; @@ -65,11 +61,10 @@ public class EvaluationContext { public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { this.jsc = jsc; this.pipeline = pipeline; - this.runtime = new SparkRuntimeContext(pipeline, jsc); + this.runtime = new SparkRuntimeContext(pipeline); } - public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, - JavaStreamingContext jssc) { + public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, JavaStreamingContext jssc) { this(jsc, pipeline); this.jssc = jssc; } @@ -192,17 +187,6 @@ public T get(PValue value) { throw new IllegalStateException("Cannot resolve un-known PObject: " + value); } - public AggregatorValues getAggregatorValues(Aggregator aggregator) - throws AggregatorRetrievalException { - return runtime.getAggregatorValues(AccumulatorSingleton.getInstance(jsc), aggregator); - } - - public T getAggregatorValue(String named, Class resultType) { - return runtime.getAggregatorValue(AccumulatorSingleton.getInstance(jsc), - named, - resultType); - } - /** * Retrieves an iterable of results associated with the PCollection passed in. * diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java index 564db39c4424..01b6b547a04b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java @@ -20,17 +20,11 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.Serializable; -import java.util.Collection; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.aggregators.NamedAggregators; -import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetricSource; -import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -43,10 +37,6 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.spark.Accumulator; -import org.apache.spark.SparkEnv$; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.metrics.MetricsSystem; - /** * The SparkRuntimeContext allows us to define useful features on the client side before our @@ -61,12 +51,11 @@ public class SparkRuntimeContext implements Serializable { private final Map> aggregators = new HashMap<>(); private transient CoderRegistry coderRegistry; - SparkRuntimeContext(Pipeline pipeline, JavaSparkContext jsc) { + SparkRuntimeContext(Pipeline pipeline) { this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions()); - registerMetrics(pipeline.getOptions().as(SparkPipelineOptions.class), jsc); } - private static String serializePipelineOptions(PipelineOptions pipelineOptions) { + private String serializePipelineOptions(PipelineOptions pipelineOptions) { try { return new ObjectMapper().writeValueAsString(pipelineOptions); } catch (JsonProcessingException e) { @@ -82,53 +71,6 @@ private static PipelineOptions deserializePipelineOptions(String serializedPipel } } - private void registerMetrics(final SparkPipelineOptions opts, final JavaSparkContext jsc) { - final Accumulator accum = AccumulatorSingleton.getInstance(jsc); - final NamedAggregators initialValue = accum.value(); - - if (opts.getEnableSparkMetricSinks()) { - final MetricsSystem metricsSystem = SparkEnv$.MODULE$.get().metricsSystem(); - final AggregatorMetricSource aggregatorMetricSource = - new AggregatorMetricSource(opts.getAppName(), initialValue); - // re-register the metrics in case of context re-use - metricsSystem.removeSource(aggregatorMetricSource); - metricsSystem.registerSource(aggregatorMetricSource); - } - } - - /** - * Retrieves corresponding value of an aggregator. - * - * @param accum The Spark Accumulator holding all Aggregators. - * @param aggregatorName Name of the aggregator to retrieve the value of. - * @param typeClass Type class of value to be retrieved. - * @param Type of object to be returned. - * @return The value of the aggregator. - */ - public T getAggregatorValue(Accumulator accum, - String aggregatorName, - Class typeClass) { - return accum.value().getValue(aggregatorName, typeClass); - } - - public AggregatorValues getAggregatorValues(Accumulator accum, - Aggregator aggregator) { - @SuppressWarnings("unchecked") - Class aggValueClass = (Class) aggregator.getCombineFn().getOutputType().getRawType(); - final T aggregatorValue = getAggregatorValue(accum, aggregator.getName(), aggValueClass); - return new AggregatorValues() { - @Override - public Collection getValues() { - return ImmutableList.of(aggregatorValue); - } - - @Override - public Map getValuesAtSteps() { - throw new UnsupportedOperationException("getValuesAtSteps is not supported."); - } - }; - } - public synchronized PipelineOptions getPipelineOptions() { return deserializePipelineOptions(serializedPipelineOptions); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 66da18131614..e033ab1255d7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -33,8 +33,8 @@ import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.beam.runners.core.AssignWindowsDoFn; import org.apache.beam.runners.spark.SparkRunner; -import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.beam.runners.spark.aggregators.SparkAggregators; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.SourceRDD; import org.apache.beam.runners.spark.io.hadoop.HadoopIO; @@ -126,7 +126,7 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { final KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); final Accumulator accum = - AccumulatorSingleton.getInstance(context.getSparkContext()); + SparkAggregators.getNamedAggregators(context.getSparkContext()); context.putDataset(transform, new BoundedDataset<>(GroupCombineFunctions.groupByKey(inRDD, accum, coder, @@ -249,7 +249,7 @@ public void evaluate(ParDo.Bound transform, EvaluationContext c final WindowFn windowFn = (WindowFn) context.getInput(transform).getWindowingStrategy().getWindowFn(); Accumulator accum = - AccumulatorSingleton.getInstance(context.getSparkContext()); + SparkAggregators.getNamedAggregators(context.getSparkContext()); Map, KV, BroadcastHelper>> sideInputs = TranslationUtils.getSideInputs(transform.getSideInputs(), context); context.putDataset(transform, @@ -281,7 +281,7 @@ public void evaluate(ParDo.BoundMulti transform, EvaluationCont final WindowFn windowFn = (WindowFn) context.getInput(transform).getWindowingStrategy().getWindowFn(); Accumulator accum = - AccumulatorSingleton.getInstance(context.getSparkContext()); + SparkAggregators.getNamedAggregators(context.getSparkContext()); JavaPairRDD, WindowedValue> all = inRDD .mapPartitionsToPair( new MultiDoFnFunction<>(accum, transform.getFn(), context.getRuntimeContext(), @@ -530,7 +530,7 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { WindowFn windowFn = (WindowFn) transform.getWindowFn(); OldDoFn addWindowsDoFn = new AssignWindowsDoFn<>(windowFn); Accumulator accum = - AccumulatorSingleton.getInstance(context.getSparkContext()); + SparkAggregators.getNamedAggregators(context.getSparkContext()); context.putDataset(transform, new BoundedDataset<>(inRDD.mapPartitions(new DoFnFunction<>(accum, addWindowsDoFn, context.getRuntimeContext(), null, null)))); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 6ed5b559e853..85d796a71cc2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -24,8 +24,8 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.core.AssignWindowsDoFn; -import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.beam.runners.spark.aggregators.SparkAggregators; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.io.SparkUnboundedSource; @@ -194,7 +194,7 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { @Override public JavaRDD> call(JavaRDD> rdd) throws Exception { final Accumulator accum = - AccumulatorSingleton.getInstance(new JavaSparkContext(rdd.context())); + SparkAggregators.getNamedAggregators(new JavaSparkContext(rdd.context())); return rdd.mapPartitions( new DoFnFunction<>(accum, addWindowsDoFn, runtimeContext, null, null)); } @@ -227,7 +227,7 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { public JavaRDD>>> call( JavaRDD>> rdd) throws Exception { final Accumulator accum = - AccumulatorSingleton.getInstance(new JavaSparkContext(rdd.context())); + SparkAggregators.getNamedAggregators(new JavaSparkContext(rdd.context())); return GroupCombineFunctions.groupByKey(rdd, accum, coder, runtimeContext, windowingStrategy); } @@ -363,7 +363,7 @@ public void evaluate(final ParDo.Bound transform, public JavaRDD> call(JavaRDD> rdd) throws Exception { final Accumulator accum = - AccumulatorSingleton.getInstance(new JavaSparkContext(rdd.context())); + SparkAggregators.getNamedAggregators(new JavaSparkContext(rdd.context())); return rdd.mapPartitions( new DoFnFunction<>(accum, transform.getFn(), runtimeContext, sideInputs, windowFn)); } @@ -396,7 +396,7 @@ public void evaluate(final ParDo.BoundMulti transform, public JavaPairRDD, WindowedValue> call( JavaRDD> rdd) throws Exception { final Accumulator accum = - AccumulatorSingleton.getInstance(new JavaSparkContext(rdd.context())); + SparkAggregators.getNamedAggregators(new JavaSparkContext(rdd.context())); return rdd.mapPartitionsToPair(new MultiDoFnFunction<>(accum, transform.getFn(), runtimeContext, transform.getMainOutputTag(), sideInputs, windowFn)); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java index 69cf1c4b37dc..54e210d96b1b 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java @@ -39,7 +39,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; @@ -52,9 +51,9 @@ */ public class SparkPipelineStateTest implements Serializable { - private static class UserException extends RuntimeException { + private static class MyCustomException extends RuntimeException { - UserException(String message) { + MyCustomException(final String message) { super(message); } } @@ -76,13 +75,13 @@ private ParDo.Bound printParDo(final String prefix) { return ParDo.of(new DoFn() { @ProcessElement - public void processElement(ProcessContext c) { + public void processElement(final ProcessContext c) { System.out.println(prefix + " " + c.element()); } }); } - private PTransform> getValues(SparkPipelineOptions options) { + private PTransform> getValues(final SparkPipelineOptions options) { return options.isStreaming() ? CreateStream.fromQueue(STREAMING_WORDS) : Create.of(BATCH_WORDS); @@ -98,7 +97,7 @@ private SparkPipelineOptions getBatchOptions() { return commonOptions.getOptions(); } - private Pipeline getPipeline(SparkPipelineOptions options) { + private Pipeline getPipeline(final SparkPipelineOptions options) { final Pipeline pipeline = Pipeline.create(options); final String name = testName.getMethodName() + "(isStreaming=" + options.isStreaming() + ")"; @@ -110,7 +109,7 @@ private Pipeline getPipeline(SparkPipelineOptions options) { return pipeline; } - private void testFailedPipeline(SparkPipelineOptions options) throws Exception { + private void testFailedPipeline(final SparkPipelineOptions options) throws Exception { SparkPipelineResult result = null; @@ -121,18 +120,17 @@ private void testFailedPipeline(SparkPipelineOptions options) throws Exception { .apply(MapElements.via(new SimpleFunction() { @Override - public String apply(String input) { - throw new UserException(FAILED_THE_BATCH_INTENTIONALLY); + public String apply(final String input) { + throw new MyCustomException(FAILED_THE_BATCH_INTENTIONALLY); } })); result = (SparkPipelineResult) pipeline.run(); result.waitUntilFinish(); - } catch (Exception e) { + } catch (final Exception e) { assertThat(e, instanceOf(Pipeline.PipelineExecutionException.class)); - assertThat(e.getCause(), instanceOf(UserCodeException.class)); - assertThat(e.getCause().getCause(), instanceOf(UserException.class)); - assertThat(e.getCause().getCause().getMessage(), is(FAILED_THE_BATCH_INTENTIONALLY)); + assertThat(e.getCause(), instanceOf(MyCustomException.class)); + assertThat(e.getCause().getMessage(), is(FAILED_THE_BATCH_INTENTIONALLY)); assertThat(result.getState(), is(PipelineResult.State.FAILED)); result.cancel(); return; @@ -141,11 +139,11 @@ public String apply(String input) { fail("An injected failure did not affect the pipeline as expected."); } - private void testTimeoutPipeline(SparkPipelineOptions options) throws Exception { + private void testTimeoutPipeline(final SparkPipelineOptions options) throws Exception { final Pipeline pipeline = getPipeline(options); - SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); + final SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); result.waitUntilFinish(Duration.millis(1)); @@ -154,22 +152,22 @@ private void testTimeoutPipeline(SparkPipelineOptions options) throws Exception result.cancel(); } - private void testCanceledPipeline(SparkPipelineOptions options) throws Exception { + private void testCanceledPipeline(final SparkPipelineOptions options) throws Exception { final Pipeline pipeline = getPipeline(options); - SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); + final SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); result.cancel(); assertThat(result.getState(), is(PipelineResult.State.CANCELLED)); } - private void testRunningPipeline(SparkPipelineOptions options) throws Exception { + private void testRunningPipeline(final SparkPipelineOptions options) throws Exception { final Pipeline pipeline = getPipeline(options); - SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); + final SparkPipelineResult result = (SparkPipelineResult) pipeline.run(); assertThat(result.getState(), is(PipelineResult.State.RUNNING)); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/ClearAggregatorsRule.java similarity index 89% rename from runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java rename to runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/ClearAggregatorsRule.java index 52ae01918328..4e91d15fe592 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/ClearAggregatorsRule.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/ClearAggregatorsRule.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.beam.runners.spark.aggregators.metrics.sink; +package org.apache.beam.runners.spark.aggregators; -import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; import org.junit.rules.ExternalResource; /** @@ -26,8 +25,13 @@ * which represents the Beam {@link org.apache.beam.sdk.transforms.Aggregator}s. */ public class ClearAggregatorsRule extends ExternalResource { + @Override protected void before() throws Throwable { + clearNamedAggregators(); + } + + public void clearNamedAggregators() { AccumulatorSingleton.clear(); } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java index 6b36bcc23a13..3b5dd21dacf2 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/NamedAggregatorsTest.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Set; import org.apache.beam.runners.spark.SparkPipelineOptions; +import org.apache.beam.runners.spark.aggregators.ClearAggregatorsRule; import org.apache.beam.runners.spark.examples.WordCount; import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptions; import org.apache.beam.sdk.Pipeline; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java index e3561d60e145..e48294526038 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/EmptyStreamAssertionTest.java @@ -23,7 +23,7 @@ import java.io.Serializable; import java.util.Collections; import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.aggregators.metrics.sink.ClearAggregatorsRule; +import org.apache.beam.runners.spark.aggregators.ClearAggregatorsRule; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming; import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptionsForStreaming; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java index e0d71d4c94c4..945ee765610e 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java @@ -29,7 +29,7 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkPipelineResult; -import org.apache.beam.runners.spark.aggregators.AccumulatorSingleton; +import org.apache.beam.runners.spark.aggregators.ClearAggregatorsRule; import org.apache.beam.runners.spark.translation.streaming.utils.EmbeddedKafkaCluster; import org.apache.beam.runners.spark.translation.streaming.utils.PAssertStreaming; import org.apache.beam.runners.spark.translation.streaming.utils.SparkTestPipelineOptionsForStreaming; @@ -83,6 +83,9 @@ public class ResumeFromCheckpointStreamingTest { public SparkTestPipelineOptionsForStreaming commonOptions = new SparkTestPipelineOptionsForStreaming(); + @Rule + public ClearAggregatorsRule clearAggregatorsRule = new ClearAggregatorsRule(); + @BeforeClass public static void init() throws IOException { EMBEDDED_ZOOKEEPER.startup(); @@ -132,8 +135,8 @@ public void testRun() throws Exception { equalTo(EXPECTED_AGG_FIRST)); } - private static SparkPipelineResult runAgain(SparkPipelineOptions options) { - AccumulatorSingleton.clear(); + private SparkPipelineResult runAgain(SparkPipelineOptions options) { + clearAggregatorsRule.clearNamedAggregators(); // sleep before next run. Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); return run(options); From 81d129597e1f3393346564f083a62dc905d3869c Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 5 Dec 2016 13:32:52 -0800 Subject: [PATCH 079/279] Update Dataflow worker to beam-master-20161205 --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 0357b4650515..f485cb8dbd3b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -210,9 +210,9 @@ public class DataflowRunner extends PipelineRunner { // Default Docker container images that execute Dataflow worker harness, residing in Google // Container Registry, separately for Batch and Streaming. public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161129"; + "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161205"; public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161129"; + "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161205"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; From 8a2f020f9781340e60609a5a8ec537871ae29570 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 30 Nov 2016 20:46:04 -0800 Subject: [PATCH 080/279] Moved KeyedWorkItem and related classes to runners-core --- .../apex/translation/operators/ApexGroupByKeyOperator.java | 4 ++-- .../main/java/org/apache/beam/runners/core/DoFnRunner.java | 1 - .../main/java/org/apache/beam/runners/core/DoFnRunners.java | 1 - .../runners/core/GroupAlsoByWindowViaWindowSetDoFn.java | 1 - .../java/org/apache/beam/runners/core}/KeyedWorkItem.java | 3 ++- .../org/apache/beam/runners/core}/KeyedWorkItemCoder.java | 4 +++- .../java/org/apache/beam/runners/core}/KeyedWorkItems.java | 3 ++- .../beam/runners/core/LateDataDroppingDoFnRunner.java | 2 -- .../java/org/apache/beam/runners/core/SplittableParDo.java | 2 -- .../apache/beam/runners/core}/KeyedWorkItemCoderTest.java | 4 +++- .../org/apache/beam/runners/core/SplittableParDoTest.java | 2 -- .../direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java | 2 +- .../org/apache/beam/runners/direct/DirectGroupByKey.java | 4 ++-- .../runners/direct/ExecutorServiceParallelExecutor.java | 4 ++-- .../runners/direct/GroupAlsoByWindowEvaluatorFactory.java | 2 +- .../beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java | 4 ++-- .../direct/SplittableProcessElementsEvaluatorFactory.java | 2 +- .../beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java | 4 ++-- .../runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java | 4 ++-- .../wrappers/streaming/SingletonKeyedWorkItem.java | 2 +- .../wrappers/streaming/SingletonKeyedWorkItemCoder.java | 4 ++-- .../translation/wrappers/streaming/WindowDoFnOperator.java | 6 ++---- .../translation/wrappers/streaming/WorkItemKeySelector.java | 2 +- 23 files changed, 31 insertions(+), 36 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/KeyedWorkItem.java (94%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/KeyedWorkItemCoder.java (97%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/KeyedWorkItems.java (97%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util => runners/core-java/src/test/java/org/apache/beam/runners/core}/KeyedWorkItemCoderTest.java (95%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index f49c7852eef7..48ac177ead7b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -42,6 +42,8 @@ import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -54,8 +56,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index c84122b63d78..aac8e8f103bc 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -20,7 +20,6 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.OldDoFn.ProcessContext; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index 38404232f2c5..da16573c3afa 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ExecutionContext.StepContext; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index 8b1081309c01..208226933eca 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java similarity index 94% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java index b273466c0b5c..c75fc2545772 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItem.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowedValue; /** * Interface that contains all the timers and elements associated with a specific work item. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index a6e3d6c10b0d..95be04732333 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; @@ -31,8 +31,10 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java index 74348428331f..94c3bb63003a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.base.MoreObjects; import com.google.common.collect.Iterables; import java.util.Collections; import java.util.Objects; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowedValue; /** * Static utility methods that provide {@link KeyedWorkItem} implementations. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 63a80d2294e9..b6f700f7a6e9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -24,8 +24,6 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 80fd17b4e748..a633111900cc 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -49,8 +49,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItemCoder; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerInternals; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java similarity index 95% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java index 1974d9e705bc..37fabddca270 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java @@ -15,14 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateNamespaces; import org.joda.time.Instant; import org.junit.Test; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index b13d83956e1a..cf96b660bea6 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -46,8 +46,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateInternals; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java index 04becd7e1a28..1fa059c0800d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.direct; +import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java index efee8016d784..21776e79880d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java @@ -20,13 +20,13 @@ import static com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItemCoder; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 929d09d8dee8..a30829520d9f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -43,13 +43,13 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.FiredTimers; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 9d25bc6a8a7b..5c6b2c19eea4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -28,6 +28,7 @@ import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; import org.apache.beam.runners.core.SystemReduceFn; @@ -44,7 +45,6 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index 4d691ea6dcf3..20d619fbc592 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -27,6 +27,8 @@ import java.util.Map; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -36,8 +38,6 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index 0eca71008abe..aae1149f99e1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -20,13 +20,13 @@ import java.util.Collection; import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.ElementAndRestriction; +import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateInternals; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java index a72681706cbd..7ba38ce728f5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java @@ -25,6 +25,8 @@ import com.google.common.collect.HashMultiset; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multiset; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -33,8 +35,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java index 3e5af14ecd0f..23340c6d361a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java @@ -25,6 +25,8 @@ import com.google.common.collect.HashMultiset; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multiset; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -33,8 +35,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java index 6d2582b5da5b..b53658e5fd7d 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import java.util.Collections; -import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index 37454a31c4bf..ad306886b856 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -26,12 +26,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.util.List; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItemCoder; import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 432dc64e2338..f2d7f1cc80c7 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -39,6 +39,8 @@ import java.util.concurrent.ScheduledFuture; import javax.annotation.Nullable; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; @@ -47,8 +49,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ExecutionContext; -import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; @@ -59,8 +59,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.DataInputView; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java index 7829163e6a26..1dff36772f2f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java @@ -18,9 +18,9 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import java.nio.ByteBuffer; +import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; From e9d835eeb6007ccc86193efbb78570cfc52e665d Mon Sep 17 00:00:00 2001 From: Sam Whittle Date: Thu, 1 Dec 2016 22:51:37 -0800 Subject: [PATCH 081/279] Deprecate TimerCallback and InMemoryTimerInternals methods using it. Instead separate advancing watermarks and removing eligible timers. --- .../beam/sdk/transforms/DoFnTester.java | 22 ++- .../util/state/InMemoryTimerInternals.java | 155 +++++++++++++----- .../beam/sdk/util/state/TimerCallback.java | 3 + .../state/InMemoryTimerInternalsTest.java | 75 ++++++--- 4 files changed, 183 insertions(+), 72 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 9f32aec9768d..43896c599404 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -52,7 +52,6 @@ import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; -import org.apache.beam.sdk.util.state.TimerCallback; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; @@ -536,19 +535,14 @@ public AggregateT getAggregatorValue(Aggregator agg) return extractAggregatorValue(agg.getName(), agg.getCombineFn()); } - private static TimerCallback collectInto(final List firedTimers) { - return new TimerCallback() { - @Override - public void onTimer(TimerInternals.TimerData timer) throws Exception { - firedTimers.add(timer); - } - }; - } - public List advanceInputWatermark(Instant newWatermark) { try { + timerInternals.advanceInputWatermark(newWatermark); final List firedTimers = new ArrayList<>(); - timerInternals.advanceInputWatermark(collectInto(firedTimers), newWatermark); + TimerInternals.TimerData timer; + while ((timer = timerInternals.removeNextEventTimer()) != null) { + firedTimers.add(timer); + } return firedTimers; } catch (Exception e) { throw new RuntimeException(e); @@ -557,8 +551,12 @@ public List advanceInputWatermark(Instant newWatermark public List advanceProcessingTime(Instant newProcessingTime) { try { + timerInternals.advanceProcessingTime(newProcessingTime); final List firedTimers = new ArrayList<>(); - timerInternals.advanceProcessingTime(collectInto(firedTimers), newProcessingTime); + TimerInternals.TimerData timer; + while ((timer = timerInternals.removeNextProcessingTimer()) != null) { + firedTimers.add(timer); + } return firedTimers; } catch (Exception e) { throw new RuntimeException(e); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index 60a90f5a07c6..60c4a960527b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -46,6 +46,9 @@ public class InMemoryTimerInternals implements TimerInternals { /** Pending processing time timers, in timestamp order. */ private PriorityQueue processingTimers = new PriorityQueue<>(11); + /** Pending synchronized processing time timers, in timestamp order. */ + private PriorityQueue synchronizedProcessingTimers = new PriorityQueue<>(11); + /** Current input watermark. */ @Nullable private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; @@ -76,9 +79,11 @@ public Instant getNextTimer(TimeDomain domain) { data = watermarkTimers.peek(); break; case PROCESSING_TIME: - case SYNCHRONIZED_PROCESSING_TIME: data = processingTimers.peek(); break; + case SYNCHRONIZED_PROCESSING_TIME: + data = synchronizedProcessingTimers.peek(); + break; default: throw new IllegalArgumentException("Unexpected time domain: " + domain); } @@ -90,8 +95,9 @@ private PriorityQueue queue(TimeDomain domain) { case EVENT_TIME: return watermarkTimers; case PROCESSING_TIME: - case SYNCHRONIZED_PROCESSING_TIME: return processingTimers; + case SYNCHRONIZED_PROCESSING_TIME: + return synchronizedProcessingTimers; default: throw new IllegalArgumentException("Unexpected time domain: " + domain); } @@ -105,7 +111,7 @@ public void setTimer(StateNamespace namespace, String timerId, Instant target, @Override public void setTimer(TimerData timerData) { - WindowTracing.trace("TestTimerInternals.setTimer: {}", timerData); + WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData); if (existingTimers.add(timerData)) { queue(timerData.getDomain()).add(timerData); } @@ -118,7 +124,7 @@ public void deleteTimer(StateNamespace namespace, String timerId) { @Override public void deleteTimer(TimerData timer) { - WindowTracing.trace("TestTimerInternals.deleteTimer: {}", timer); + WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer); existingTimers.remove(timer); queue(timer.getDomain()).remove(timer); } @@ -144,15 +150,15 @@ public String toString() { return MoreObjects.toStringHelper(getClass()) .add("watermarkTimers", watermarkTimers) .add("processingTimers", processingTimers) + .add("synchronizedProcessingTimers", synchronizedProcessingTimers) .add("inputWatermarkTime", inputWatermarkTime) .add("outputWatermarkTime", outputWatermarkTime) .add("processingTime", processingTime) .toString(); } - /** Advances input watermark to the given value and fires event-time timers accordingly. */ - public void advanceInputWatermark( - TimerCallback timerCallback, Instant newInputWatermark) throws Exception { + /** Advances input watermark to the given value. */ + public void advanceInputWatermark(Instant newInputWatermark) throws Exception { checkNotNull(newInputWatermark); checkState( !newInputWatermark.isBefore(inputWatermarkTime), @@ -160,11 +166,9 @@ public void advanceInputWatermark( inputWatermarkTime, newInputWatermark); WindowTracing.trace( - "TestTimerInternals.advanceInputWatermark: from {} to {}", - inputWatermarkTime, - newInputWatermark); + "{}.advanceInputWatermark: from {} to {}", + getClass().getSimpleName(), inputWatermarkTime, newInputWatermark); inputWatermarkTime = newInputWatermark; - advanceAndFire(timerCallback, newInputWatermark, TimeDomain.EVENT_TIME); } /** Advances output watermark to the given value. */ @@ -173,9 +177,8 @@ public void advanceOutputWatermark(Instant newOutputWatermark) { final Instant adjustedOutputWatermark; if (newOutputWatermark.isAfter(inputWatermarkTime)) { WindowTracing.trace( - "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}", - newOutputWatermark, - inputWatermarkTime); + "{}.advanceOutputWatermark: clipping output watermark from {} to {}", + getClass().getSimpleName(), newOutputWatermark, inputWatermarkTime); adjustedOutputWatermark = inputWatermarkTime; } else { adjustedOutputWatermark = newOutputWatermark; @@ -187,60 +190,134 @@ public void advanceOutputWatermark(Instant newOutputWatermark) { outputWatermarkTime, adjustedOutputWatermark); WindowTracing.trace( - "TestTimerInternals.advanceOutputWatermark: from {} to {}", - outputWatermarkTime, - adjustedOutputWatermark); + "{}.advanceOutputWatermark: from {} to {}", + getClass().getSimpleName(), outputWatermarkTime, adjustedOutputWatermark); outputWatermarkTime = adjustedOutputWatermark; } - /** Advances processing time to the given value and fires processing-time timers accordingly. */ - public void advanceProcessingTime( - TimerCallback timerCallback, Instant newProcessingTime) throws Exception { + /** Advances processing time to the given value. */ + public void advanceProcessingTime(Instant newProcessingTime) throws Exception { checkState( !newProcessingTime.isBefore(processingTime), "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime); WindowTracing.trace( - "TestTimerInternals.advanceProcessingTime: from {} to {}", - processingTime, - newProcessingTime); + "{}.advanceProcessingTime: from {} to {}", + getClass().getSimpleName(), processingTime, newProcessingTime); processingTime = newProcessingTime; + } + + /** Advances synchronized processing time to the given value. */ + public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime) + throws Exception { + checkState( + !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime), + "Cannot move processing time backwards from %s to %s", + synchronizedProcessingTime, + newSynchronizedProcessingTime); + WindowTracing.trace( + "{}.advanceProcessingTime: from {} to {}", + getClass().getSimpleName(), synchronizedProcessingTime, newSynchronizedProcessingTime); + synchronizedProcessingTime = newSynchronizedProcessingTime; + } + + /** Returns the next eligible event time timer, if none returns null. */ + @Nullable + public TimerData removeNextEventTimer() { + TimerData timer = removeNextTimer(inputWatermarkTime, TimeDomain.EVENT_TIME); + if (timer != null) { + WindowTracing.trace( + "{}.removeNextEventTimer: firing {} at {}", + getClass().getSimpleName(), timer, inputWatermarkTime); + } + return timer; + } + + /** Returns the next eligible processing time timer, if none returns null. */ + @Nullable + public TimerData removeNextProcessingTimer() { + TimerData timer = removeNextTimer(processingTime, TimeDomain.PROCESSING_TIME); + if (timer != null) { + WindowTracing.trace( + "{}.removeNextProcessingTimer: firing {} at {}", + getClass().getSimpleName(), timer, processingTime); + } + return timer; + } + + /** Returns the next eligible synchronized processing time timer, if none returns null. */ + @Nullable + public TimerData removeNextSynchronizedProcessingTimer() { + TimerData timer = removeNextTimer( + synchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); + if (timer != null) { + WindowTracing.trace( + "{}.removeNextSynchronizedProcessingTimer: firing {} at {}", + getClass().getSimpleName(), timer, synchronizedProcessingTime); + } + return timer; + } + + @Nullable + private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) { + PriorityQueue queue = queue(domain); + if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { + TimerData timer = queue.remove(); + existingTimers.remove(timer); + return timer; + } else { + return null; + } + } + + /** Advances input watermark to the given value and fires event-time timers accordingly. + * + * @deprecated Use advanceInputWatermark without callback and fireEventTimers. + */ + @Deprecated + public void advanceInputWatermark( + TimerCallback timerCallback, Instant newInputWatermark) throws Exception { + advanceInputWatermark(newInputWatermark); + advanceAndFire(timerCallback, newInputWatermark, TimeDomain.EVENT_TIME); + } + + /** Advances processing time to the given value and fires processing-time timers accordingly. + * + * @deprecated Use advanceProcessingTime without callback and fireProcessingTimers. + */ + @Deprecated + public void advanceProcessingTime( + TimerCallback timerCallback, Instant newProcessingTime) throws Exception { + advanceProcessingTime(newProcessingTime); advanceAndFire(timerCallback, newProcessingTime, TimeDomain.PROCESSING_TIME); } /** * Advances synchronized processing time to the given value and fires processing-time timers * accordingly. + * + * @deprecated Use advanceInputWatermark without callback and fireSynchronizedProcessingTimers. */ + @Deprecated public void advanceSynchronizedProcessingTime( TimerCallback timerCallback, Instant newSynchronizedProcessingTime) throws Exception { - checkState( - !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime), - "Cannot move processing time backwards from %s to %s", - processingTime, - newSynchronizedProcessingTime); - WindowTracing.trace( - "TestTimerInternals.advanceProcessingTime: from {} to {}", - synchronizedProcessingTime, - newSynchronizedProcessingTime); - synchronizedProcessingTime = newSynchronizedProcessingTime; + advanceSynchronizedProcessingTime(newSynchronizedProcessingTime); advanceAndFire( timerCallback, newSynchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); } + @Deprecated private void advanceAndFire( TimerCallback timerCallback, Instant currentTime, TimeDomain domain) throws Exception { checkNotNull(timerCallback); - PriorityQueue queue = queue(domain); - while (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { - // Remove before firing, so that if the callback adds another identical - // timer we don't remove it. - TimerData timer = queue.remove(); + TimerData timer; + while ((timer = removeNextTimer(currentTime, domain)) != null) { WindowTracing.trace( - "InMemoryTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime); + "{}.advanceAndFire: firing {} at {}", + getClass().getSimpleName(), timer, currentTime); timerCallback.onTimer(timer); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java index 6598e300f256..1d68e367f5dd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java @@ -21,7 +21,10 @@ /** * A callback that processes a {@link TimerInternals.TimerData TimerData}. + * + * @deprecated Use TimerInternals.advanceTime and removeTimers instead of callback. */ +@Deprecated public interface TimerCallback { /** Processes the {@link TimerInternals.TimerData TimerData}. */ void onTimer(TimerInternals.TimerData timer) throws Exception; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java index 951803a8f659..87c9aea3fba0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.util.state; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.times; + import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.joda.time.Instant; @@ -53,6 +57,34 @@ public void testFiringTimers() throws Exception { underTest.setTimer(processingTime1); underTest.setTimer(processingTime2); + underTest.advanceProcessingTime(new Instant(20)); + assertEquals(processingTime1, underTest.removeNextProcessingTimer()); + assertNull(underTest.removeNextProcessingTimer()); + + // Advancing just a little shouldn't refire + underTest.advanceProcessingTime(new Instant(21)); + assertNull(underTest.removeNextProcessingTimer()); + + // Adding the timer and advancing a little should refire + underTest.setTimer(processingTime1); + assertEquals(processingTime1, underTest.removeNextProcessingTimer()); + assertNull(underTest.removeNextProcessingTimer()); + + // And advancing the rest of the way should still have the other timer + underTest.advanceProcessingTime(new Instant(30)); + assertEquals(processingTime2, underTest.removeNextProcessingTimer()); + assertNull(underTest.removeNextProcessingTimer()); + } + + @Test + public void testFiringTimersWithCallback() throws Exception { + InMemoryTimerInternals underTest = new InMemoryTimerInternals(); + TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME); + TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME); + + underTest.setTimer(processingTime1); + underTest.setTimer(processingTime2); + underTest.advanceProcessingTime(timerCallback, new Instant(20)); Mockito.verify(timerCallback).onTimer(processingTime1); Mockito.verifyNoMoreInteractions(timerCallback); @@ -63,8 +95,8 @@ public void testFiringTimers() throws Exception { // Adding the timer and advancing a little should refire underTest.setTimer(processingTime1); - Mockito.verify(timerCallback).onTimer(processingTime1); underTest.advanceProcessingTime(timerCallback, new Instant(21)); + Mockito.verify(timerCallback, times(2)).onTimer(processingTime1); Mockito.verifyNoMoreInteractions(timerCallback); // And advancing the rest of the way should still have the other timer @@ -76,41 +108,42 @@ public void testFiringTimers() throws Exception { @Test public void testTimerOrdering() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); - TimerData watermarkTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME); + TimerData eventTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME); TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME); - TimerData watermarkTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME); + TimerData eventTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME); TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME); underTest.setTimer(processingTime1); - underTest.setTimer(watermarkTime1); + underTest.setTimer(eventTime1); underTest.setTimer(processingTime2); - underTest.setTimer(watermarkTime2); + underTest.setTimer(eventTime2); - underTest.advanceInputWatermark(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimer(watermarkTime1); - Mockito.verify(timerCallback).onTimer(watermarkTime2); - Mockito.verifyNoMoreInteractions(timerCallback); + underTest.advanceInputWatermark(new Instant(30)); + assertEquals(eventTime1, underTest.removeNextEventTimer()); + assertEquals(eventTime2, underTest.removeNextEventTimer()); + assertNull(underTest.removeNextEventTimer()); - underTest.advanceProcessingTime(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimer(processingTime1); - Mockito.verify(timerCallback).onTimer(processingTime2); - Mockito.verifyNoMoreInteractions(timerCallback); + underTest.advanceProcessingTime(new Instant(30)); + assertEquals(processingTime1, underTest.removeNextProcessingTimer()); + assertEquals(processingTime2, underTest.removeNextProcessingTimer()); + assertNull(underTest.removeNextProcessingTimer()); } @Test public void testDeduplicate() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); - TimerData watermarkTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME); + TimerData eventTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME); TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME); - underTest.setTimer(watermarkTime); - underTest.setTimer(watermarkTime); + underTest.setTimer(eventTime); + underTest.setTimer(eventTime); underTest.setTimer(processingTime); underTest.setTimer(processingTime); - underTest.advanceProcessingTime(timerCallback, new Instant(20)); - underTest.advanceInputWatermark(timerCallback, new Instant(20)); + underTest.advanceProcessingTime(new Instant(20)); + underTest.advanceInputWatermark(new Instant(20)); - Mockito.verify(timerCallback).onTimer(processingTime); - Mockito.verify(timerCallback).onTimer(watermarkTime); - Mockito.verifyNoMoreInteractions(timerCallback); + assertEquals(processingTime, underTest.removeNextProcessingTimer()); + assertNull(underTest.removeNextProcessingTimer()); + assertEquals(eventTime, underTest.removeNextEventTimer()); + assertNull(underTest.removeNextEventTimer()); } } From 3434e8a434dea2f0ba5bb6e561bb7f3bd9d5d603 Mon Sep 17 00:00:00 2001 From: Sam Whittle Date: Mon, 5 Dec 2016 14:57:29 -0800 Subject: [PATCH 082/279] Add test for InMemoryTimerInternals synchronized processing time timers. Ensure that processing time and synchronized processing timer are not null. --- .../sdk/util/state/InMemoryTimerInternals.java | 8 +++++--- .../apache/beam/sdk/util/state/TimerCallback.java | 2 +- .../sdk/util/state/InMemoryTimerInternalsTest.java | 14 ++++++++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index 60c4a960527b..159b5830b642 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -50,7 +50,7 @@ public class InMemoryTimerInternals implements TimerInternals { private PriorityQueue synchronizedProcessingTimers = new PriorityQueue<>(11); /** Current input watermark. */ - @Nullable private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; /** Current output watermark. */ @Nullable private Instant outputWatermarkTime = null; @@ -59,7 +59,7 @@ public class InMemoryTimerInternals implements TimerInternals { private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; /** Current synchronized processing time. */ - @Nullable private Instant synchronizedProcessingTime = null; + private Instant synchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; @Override @Nullable @@ -142,7 +142,7 @@ public Instant currentSynchronizedProcessingTime() { @Override public Instant currentInputWatermarkTime() { - return checkNotNull(inputWatermarkTime); + return inputWatermarkTime; } @Override @@ -197,6 +197,7 @@ public void advanceOutputWatermark(Instant newOutputWatermark) { /** Advances processing time to the given value. */ public void advanceProcessingTime(Instant newProcessingTime) throws Exception { + checkNotNull(newProcessingTime); checkState( !newProcessingTime.isBefore(processingTime), "Cannot move processing time backwards from %s to %s", @@ -211,6 +212,7 @@ public void advanceProcessingTime(Instant newProcessingTime) throws Exception { /** Advances synchronized processing time to the given value. */ public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime) throws Exception { + checkNotNull(newSynchronizedProcessingTime); checkState( !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime), "Cannot move processing time backwards from %s to %s", diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java index 1d68e367f5dd..83791d6fade0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java @@ -22,7 +22,7 @@ /** * A callback that processes a {@link TimerInternals.TimerData TimerData}. * - * @deprecated Use TimerInternals.advanceTime and removeTimers instead of callback. + * @deprecated Use InMemoryTimerInternals advance and remove methods instead of callback. */ @Deprecated public interface TimerCallback { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java index 87c9aea3fba0..1e42864bf3ed 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java @@ -110,23 +110,37 @@ public void testTimerOrdering() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); TimerData eventTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME); TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME); + TimerData synchronizedProcessingTime1 = TimerData.of( + NS1, new Instant(19), TimeDomain.SYNCHRONIZED_PROCESSING_TIME); TimerData eventTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME); TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME); + TimerData synchronizedProcessingTime2 = TimerData.of( + NS1, new Instant(29), TimeDomain.SYNCHRONIZED_PROCESSING_TIME); underTest.setTimer(processingTime1); underTest.setTimer(eventTime1); + underTest.setTimer(synchronizedProcessingTime1); underTest.setTimer(processingTime2); underTest.setTimer(eventTime2); + underTest.setTimer(synchronizedProcessingTime2); + assertNull(underTest.removeNextEventTimer()); underTest.advanceInputWatermark(new Instant(30)); assertEquals(eventTime1, underTest.removeNextEventTimer()); assertEquals(eventTime2, underTest.removeNextEventTimer()); assertNull(underTest.removeNextEventTimer()); + assertNull(underTest.removeNextProcessingTimer()); underTest.advanceProcessingTime(new Instant(30)); assertEquals(processingTime1, underTest.removeNextProcessingTimer()); assertEquals(processingTime2, underTest.removeNextProcessingTimer()); assertNull(underTest.removeNextProcessingTimer()); + + assertNull(underTest.removeNextSynchronizedProcessingTimer()); + underTest.advanceSynchronizedProcessingTime(new Instant(30)); + assertEquals(synchronizedProcessingTime1, underTest.removeNextSynchronizedProcessingTimer()); + assertEquals(synchronizedProcessingTime2, underTest.removeNextSynchronizedProcessingTimer()); + assertNull(underTest.removeNextProcessingTimer()); } @Test From 41ae08bf18525f52b03252dee783505ae400911e Mon Sep 17 00:00:00 2001 From: Vladisav Jelisavcic Date: Sun, 4 Dec 2016 10:42:28 +0100 Subject: [PATCH 083/279] [BEAM-961] Add starting number to CountingInput --- .../org/apache/beam/sdk/io/CountingInput.java | 42 +++++++++++++++---- .../apache/beam/sdk/io/CountingSource.java | 11 +++++ .../apache/beam/sdk/io/CountingInputTest.java | 42 ++++++++++++++----- 3 files changed, 76 insertions(+), 19 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java index f47921529644..456d291ef35a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java @@ -35,7 +35,7 @@ /** * A {@link PTransform} that produces longs. When used to produce a * {@link IsBounded#BOUNDED bounded} {@link PCollection}, {@link CountingInput} starts at {@code 0} - * and counts up to a specified maximum. When used to produce an + * or starting value, and counts up to a specified maximum. When used to produce an * {@link IsBounded#UNBOUNDED unbounded} {@link PCollection}, it counts up to {@link Long#MAX_VALUE} * and then never produces more output. (In practice, this limit should never be reached.) * @@ -43,7 +43,8 @@ * {@link OffsetBasedSource.OffsetBasedReader}, so it performs efficient initial splitting and it * supports dynamic work rebalancing. * - *

    To produce a bounded {@code PCollection}, use {@link CountingInput#upTo(long)}: + *

    To produce a bounded {@code PCollection} starting from {@code 0}, + * use {@link CountingInput#upTo(long)}: * *

    {@code
      * Pipeline p = ...
    @@ -51,6 +52,9 @@
      * PCollection bounded = p.apply(producer);
      * }
    * + *

    To produce a bounded {@code PCollection} starting from {@code startOffset}, + * use {@link CountingInput#forSubrange(long, long)} instead. + * *

    To produce an unbounded {@code PCollection}, use {@link CountingInput#unbounded()}, * calling {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to provide values * with timestamps other than {@link Instant#now}. @@ -75,6 +79,16 @@ public static BoundedCountingInput upTo(long numElements) { return new BoundedCountingInput(numElements); } + /** + * Creates a {@link BoundedCountingInput} that will produce elements + * starting from {@code startIndex} to {@code endIndex - 1}. + */ + public static BoundedCountingInput forSubrange(long startIndex, long endIndex) { + checkArgument(endIndex > startIndex, "endIndex (%s) must be greater than startIndex (%s)", + endIndex, startIndex); + return new BoundedCountingInput(startIndex, endIndex); + } + /** * Creates an {@link UnboundedCountingInput} that will produce numbers starting from {@code 0} up * to {@link Long#MAX_VALUE}. @@ -102,23 +116,35 @@ public static UnboundedCountingInput unbounded() { * 0. */ public static class BoundedCountingInput extends PTransform> { - private final long numElements; + private final long startIndex; + private final long endIndex; private BoundedCountingInput(long numElements) { - this.numElements = numElements; + this.endIndex = numElements; + this.startIndex = 0; + } + + private BoundedCountingInput(long startIndex, long endIndex) { + this.endIndex = endIndex; + this.startIndex = startIndex; } - @SuppressWarnings("deprecation") @Override public PCollection apply(PBegin begin) { - return begin.apply(Read.from(CountingSource.upTo(numElements))); + return begin.apply(Read.from(CountingSource.createSourceForSubrange(startIndex, endIndex))); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("upTo", numElements) - .withLabel("Count Up To")); + + if (startIndex == 0) { + builder.add(DisplayData.item("upTo", endIndex) + .withLabel("Count Up To")); + } else { + builder.add(DisplayData.item("startAt", startIndex).withLabel("Count Starting At")) + .add(DisplayData.item("upTo", endIndex).withLabel("Count Up To")); + } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index 59a8df8ac9d9..bc7fb789b561 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -82,6 +82,17 @@ public static BoundedSource upTo(long numElements) { return new BoundedCountingSource(0, numElements); } + /** + * Creates a {@link BoundedSource} that will produce elements + * from {@code startIndex} to {@code endIndex - 1}. + */ + static BoundedSource createSourceForSubrange(long startIndex, long endIndex) { + checkArgument(endIndex > startIndex, "endIndex (%s) must be greater than startIndex (%s)", + endIndex, startIndex); + + return new BoundedCountingSource(startIndex, endIndex); + } + /** * Create a new {@link UnboundedCountingSource}. */ 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 2397d10fe4f1..02b4ba072851 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 @@ -49,21 +49,21 @@ */ @RunWith(JUnit4.class) public class CountingInputTest { - public static void addCountingAsserts(PCollection input, long numElements) { + public static void addCountingAsserts(PCollection input, long start, long end) { // Count == numElements PAssert.thatSingleton(input.apply("Count", Count.globally())) - .isEqualTo(numElements); + .isEqualTo(end - start); // Unique count == numElements PAssert.thatSingleton( input .apply(Distinct.create()) .apply("UniqueCount", Count.globally())) - .isEqualTo(numElements); - // Min == 0 - PAssert.thatSingleton(input.apply("Min", Min.globally())).isEqualTo(0L); - // Max == numElements-1 + .isEqualTo(end - start); + // Min == start + PAssert.thatSingleton(input.apply("Min", Min.globally())).isEqualTo(start); + // Max == end-1 PAssert.thatSingleton(input.apply("Max", Max.globally())) - .isEqualTo(numElements - 1); + .isEqualTo(end - 1); } @Test @@ -73,7 +73,19 @@ public void testBoundedInput() { long numElements = 1000; PCollection input = p.apply(CountingInput.upTo(numElements)); - addCountingAsserts(input, numElements); + addCountingAsserts(input, 0, numElements); + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testBoundedInputSubrange() { + Pipeline p = TestPipeline.create(); + long start = 10; + long end = 1000; + PCollection input = p.apply(CountingInput.forSubrange(start, end)); + + addCountingAsserts(input, start, end); p.run(); } @@ -84,6 +96,14 @@ public void testBoundedDisplayData() { assertThat(displayData, hasDisplayItem("upTo", 1234)); } + @Test + public void testBoundedDisplayDataSubrange() { + PTransform input = CountingInput.forSubrange(12, 1234); + DisplayData displayData = DisplayData.from(input); + assertThat(displayData, hasDisplayItem("startAt", 12)); + assertThat(displayData, hasDisplayItem("upTo", 1234)); + } + @Test @Category(RunnableOnService.class) public void testUnboundedInput() { @@ -92,7 +112,7 @@ public void testUnboundedInput() { PCollection input = p.apply(CountingInput.unbounded().withMaxNumRecords(numElements)); - addCountingAsserts(input, numElements); + addCountingAsserts(input, 0, numElements); p.run(); } @@ -110,7 +130,7 @@ public void testUnboundedInputRate() { .withRate(elemsPerPeriod, periodLength) .withMaxNumRecords(numElements)); - addCountingAsserts(input, numElements); + addCountingAsserts(input, 0, numElements); long expectedRuntimeMillis = (periodLength.getMillis() * numElements) / elemsPerPeriod; Instant startTime = Instant.now(); p.run(); @@ -136,7 +156,7 @@ public void testUnboundedInputTimestamps() { CountingInput.unbounded() .withTimestampFn(new ValueAsTimestampFn()) .withMaxNumRecords(numElements)); - addCountingAsserts(input, numElements); + addCountingAsserts(input, 0, numElements); PCollection diffs = input From a37068dd96aaf2f8b8aa8e86f802dd5da8ee8381 Mon Sep 17 00:00:00 2001 From: Neelesh Srinivas Salian Date: Tue, 6 Dec 2016 03:58:01 -0800 Subject: [PATCH 084/279] BEAM-651: Rename setTypeDescriptorInternal to setTypeDescriptor --- .../java/org/apache/beam/runners/core/SplittableParDo.java | 2 +- .../runners/direct/ParDoSingleViaMultiOverrideFactory.java | 2 +- .../runners/dataflow/transforms/DataflowGroupByKeyTest.java | 2 +- .../beam/runners/dataflow/transforms/DataflowViewTest.java | 2 +- .../src/main/java/org/apache/beam/sdk/transforms/ParDo.java | 4 ++-- .../src/main/java/org/apache/beam/sdk/values/PCollection.java | 4 ++-- .../java/org/apache/beam/sdk/values/PCollectionTuple.java | 2 +- .../src/main/java/org/apache/beam/sdk/values/TypedPValue.java | 2 +- .../java/org/apache/beam/sdk/transforms/GroupByKeyTest.java | 2 +- .../test/java/org/apache/beam/sdk/transforms/ViewTest.java | 2 +- 10 files changed, 12 insertions(+), 12 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index a633111900cc..78f373bfd5aa 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -259,7 +259,7 @@ public PCollectionTuple apply( input.isBounded().and(signature.isBoundedPerElement())); // Set output type descriptor similarly to how ParDo.BoundMulti does it. - outputs.get(mainOutputTag).setTypeDescriptorInternal(fn.getOutputTypeDescriptor()); + outputs.get(mainOutputTag).setTypeDescriptor(fn.getOutputTypeDescriptor()); return outputs; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java index f220a4617f60..7f2de667bb60 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java @@ -61,7 +61,7 @@ public PCollection apply(PCollection input) { .withOutputTags(mainOutputTag, TupleTagList.empty())); PCollection output = outputs.get(mainOutputTag); - output.setTypeDescriptorInternal(underlyingParDo.getNewFn().getOutputTypeDescriptor()); + output.setTypeDescriptor(underlyingParDo.getNewFn().getOutputTypeDescriptor()); return output; } } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java index 6d5a63ab44d7..bb84d9855054 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java @@ -97,7 +97,7 @@ public PCollection> apply(PBegin input) { input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptorInternal(new TypeDescriptor>() {}); + .setTypeDescriptor(new TypeDescriptor>() {}); } }); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java index 2eddcf78c22f..ed3f2cd1c89f 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java @@ -86,7 +86,7 @@ public PCollection> apply(PBegin input) { input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptorInternal(new TypeDescriptor>() {}); + .setTypeDescriptor(new TypeDescriptor>() {}); } }) .apply(view); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 9453294cb832..ba6e64423702 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -803,7 +803,7 @@ public PCollection apply(PCollection input) { input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setTypeDescriptorInternal(getOldFn().getOutputTypeDescriptor()); + .setTypeDescriptor(getOldFn().getOutputTypeDescriptor()); } @Override @@ -1065,7 +1065,7 @@ public PCollectionTuple apply(PCollection input) { // The fn will likely be an instance of an anonymous subclass // such as DoFn { }, thus will have a high-fidelity // TypeDescriptor for the output type. - outputs.get(mainOutputTag).setTypeDescriptorInternal(getOldFn().getOutputTypeDescriptor()); + outputs.get(mainOutputTag).setTypeDescriptor(getOldFn().getOutputTypeDescriptor()); return outputs; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java index ca3f0ba4b526..b69185b73ff1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java @@ -202,8 +202,8 @@ private PCollection(Pipeline p) { * etc., to provide more detailed reflective information. */ @Override - public PCollection setTypeDescriptorInternal(TypeDescriptor typeDescriptor) { - super.setTypeDescriptorInternal(typeDescriptor); + public PCollection setTypeDescriptor(TypeDescriptor typeDescriptor) { + super.setTypeDescriptor(typeDescriptor); return this; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java index f6776f07979a..727d882d745e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java @@ -219,7 +219,7 @@ public static PCollectionTuple ofPrimitiveOutputsInternal( TypeDescriptor token = (TypeDescriptor) outputTag.getTypeDescriptor(); PCollection outputCollection = PCollection .createPrimitiveOutputInternal(pipeline, windowingStrategy, isBounded) - .setTypeDescriptorInternal(token); + .setTypeDescriptor(token); pcollectionMap.put(outputTag, outputCollection); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java index 0387f0a69d8c..7afd0a1fb8ca 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypedPValue.java @@ -119,7 +119,7 @@ public TypeDescriptor getTypeDescriptor() { * reflective type information will lead to better {@link Coder} * inference. */ - public TypedPValue setTypeDescriptorInternal(TypeDescriptor typeDescriptor) { + public TypedPValue setTypeDescriptor(TypeDescriptor typeDescriptor) { this.typeDescriptor = typeDescriptor; return this; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index bea0e2d90185..535ffec76bec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -311,7 +311,7 @@ public PCollection> apply(PBegin input) { input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptorInternal(new TypeDescriptor>() {}); + .setTypeDescriptor(new TypeDescriptor>() {}); } }); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index 69b618607057..fe2d1251295a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -1365,7 +1365,7 @@ public PCollection> apply(PBegin input) { input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptorInternal(new TypeDescriptor>() {}); + .setTypeDescriptor(new TypeDescriptor>() {}); } }) .apply(view); From 6afe91b1411815c44b951773b443a011ba4cbf31 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 6 Dec 2016 06:20:07 +0100 Subject: [PATCH 085/279] [BEAM-293] StreamingOptions doesn't extend GcpOptions --- .../main/java/org/apache/beam/sdk/options/StreamingOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java index a9031cb69dc9..99ecd8fbdb44 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/StreamingOptions.java @@ -21,7 +21,7 @@ * Options used to configure streaming. */ public interface StreamingOptions extends - ApplicationNameOptions, GcpOptions, PipelineOptions { + ApplicationNameOptions, PipelineOptions { /** * Set to true if running a streaming pipeline. */ From d6c6ad37149622e4d35af39727cdf774e6263d1e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 2 Dec 2016 10:56:36 -0800 Subject: [PATCH 086/279] Add DirectGraphs to DirectRunner Tests Add getGraph(Pipeline) and getProducer(PValue), which use the DirectGraphVisitor and DirectGraph methods to provide access to the producing AppliedPTransform. Remove getProducingTransformInternal from everywhere except DirectGraphVisitorTest --- .../BoundedReadEvaluatorFactoryTest.java | 18 +- .../direct/DirectGraphVisitorTest.java | 1 + .../beam/runners/direct/DirectGraphs.java | 35 +++ .../runners/direct/EvaluationContextTest.java | 82 +++--- .../direct/FlattenEvaluatorFactoryTest.java | 15 +- .../GroupByKeyEvaluatorFactoryTest.java | 2 +- .../GroupByKeyOnlyEvaluatorFactoryTest.java | 3 +- .../ImmutabilityEnforcementFactoryTest.java | 2 +- .../runners/direct/ParDoEvaluatorTest.java | 3 +- .../StatefulParDoEvaluatorFactoryTest.java | 4 +- .../direct/StepTransformResultTest.java | 2 +- .../TestStreamEvaluatorFactoryTest.java | 14 +- .../runners/direct/TransformExecutorTest.java | 9 +- .../UnboundedReadEvaluatorFactoryTest.java | 24 +- .../direct/ViewEvaluatorFactoryTest.java | 4 +- .../direct/WatermarkCallbackExecutorTest.java | 6 +- .../runners/direct/WatermarkManagerTest.java | 237 ++++++++---------- 17 files changed, 246 insertions(+), 215 deletions(-) create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index b1ff6890b19a..acb14441b407 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -80,6 +80,7 @@ public class BoundedReadEvaluatorFactoryTest { private BoundedReadEvaluatorFactory factory; @Mock private EvaluationContext context; private BundleFactory bundleFactory; + private AppliedPTransform longsProducer; @Before public void setup() { @@ -92,6 +93,7 @@ public void setup() { new BoundedReadEvaluatorFactory( context, Long.MAX_VALUE /* minimum size for dynamic splits */); bundleFactory = ImmutableListBundleFactory.create(); + longsProducer = DirectGraphs.getProducer(longs); } @Test @@ -102,11 +104,11 @@ public void boundedSourceInMemoryTransformEvaluatorProducesElements() throws Exc Collection> initialInputs = new BoundedReadEvaluatorFactory.InputProvider(context) - .getInitialInputs(longs.getProducingTransformInternal(), 1); + .getInitialInputs(longsProducer, 1); List> outputs = new ArrayList<>(); for (CommittedBundle shardBundle : initialInputs) { TransformEvaluator evaluator = - factory.forApplication(longs.getProducingTransformInternal(), null); + factory.forApplication(longsProducer, null); for (WindowedValue shard : shardBundle.getElements()) { evaluator.processElement((WindowedValue) shard); } @@ -141,7 +143,7 @@ public void boundedSourceEvaluatorProducesDynamicSplits() throws Exception { } PCollection read = TestPipeline.create().apply(Read.from(new TestSource<>(VarLongCoder.of(), 5, elems))); - AppliedPTransform transform = read.getProducingTransformInternal(); + AppliedPTransform transform = DirectGraphs.getProducer(read); Collection> unreadInputs = new BoundedReadEvaluatorFactory.InputProvider(context).getInitialInputs(transform, 1); @@ -191,7 +193,7 @@ public void boundedSourceEvaluatorDynamicSplitsUnsplittable() throws Exception { PCollection read = TestPipeline.create() .apply(Read.from(SourceTestUtils.toUnsplittableSource(CountingSource.upTo(10L)))); - AppliedPTransform transform = read.getProducingTransformInternal(); + AppliedPTransform transform = DirectGraphs.getProducer(read); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); @@ -238,7 +240,7 @@ public UncommittedBundle answer(InvocationOnMock invocation) throws Throwable }); Collection> initialInputs = new BoundedReadEvaluatorFactory.InputProvider(context) - .getInitialInputs(longs.getProducingTransformInternal(), 3); + .getInitialInputs(longsProducer, 3); assertThat(initialInputs, hasSize(allOf(greaterThanOrEqualTo(3), lessThanOrEqualTo(4)))); @@ -271,7 +273,7 @@ public void boundedSourceInMemoryTransformEvaluatorShardsOfSource() throws Excep CommittedBundle> shards = rootBundle.commit(Instant.now()); TransformEvaluator> evaluator = - factory.forApplication(longs.getProducingTransformInternal(), shards); + factory.forApplication(longsProducer, shards); for (WindowedValue> shard : shards.getElements()) { UncommittedBundle outputBundle = bundleFactory.createBundle(longs); when(context.createBundle(longs)).thenReturn(outputBundle); @@ -299,7 +301,7 @@ public void boundedSourceEvaluatorClosesReader() throws Exception { TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); - AppliedPTransform sourceTransform = pcollection.getProducingTransformInternal(); + AppliedPTransform sourceTransform = DirectGraphs.getProducer(pcollection); UncommittedBundle output = bundleFactory.createBundle(pcollection); when(context.createBundle(pcollection)).thenReturn(output); @@ -320,7 +322,7 @@ public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception { TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); - AppliedPTransform sourceTransform = pcollection.getProducingTransformInternal(); + AppliedPTransform sourceTransform = DirectGraphs.getProducer(pcollection); UncommittedBundle output = bundleFactory.createBundle(pcollection); when(context.createBundle(pcollection)).thenReturn(output); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java index d218a81e2dc2..fb84de8a4ad3 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java @@ -48,6 +48,7 @@ /** * Tests for {@link DirectGraphVisitor}. */ +// TODO: Replace uses of getProducing @RunWith(JUnit4.class) public class DirectGraphVisitorTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java new file mode 100644 index 000000000000..73ada196104f --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java @@ -0,0 +1,35 @@ +/* + * 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.direct; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.values.PValue; + +/** Test utilities for the {@link DirectRunner}. */ +final class DirectGraphs { + public static DirectGraph getGraph(Pipeline p) { + DirectGraphVisitor visitor = new DirectGraphVisitor(); + p.traverseTopologically(visitor); + return visitor.getGraph(); + } + + public static AppliedPTransform getProducer(PValue value) { + return getGraph(value.getPipeline()).getProducer(value); + } +} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 17cdea1440a2..a2bb15ecb037 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -86,9 +86,13 @@ public class EvaluationContextTest { private PCollectionView> view; private PCollection unbounded; - private BundleFactory bundleFactory; private DirectGraph graph; + private AppliedPTransform createdProducer; + private AppliedPTransform downstreamProducer; + private AppliedPTransform viewProducer; + private AppliedPTransform unboundedProducer; + @Before public void setup() { DirectRunner runner = @@ -101,14 +105,16 @@ public void setup() { view = created.apply(View.asIterable()); unbounded = p.apply(CountingInput.unbounded()); - DirectGraphVisitor graphVisitor = new DirectGraphVisitor(); - p.traverseTopologically(graphVisitor); - - bundleFactory = ImmutableListBundleFactory.create(); - graph = graphVisitor.getGraph(); + BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + graph = DirectGraphs.getGraph(p); context = EvaluationContext.create( runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph); + + createdProducer = graph.getProducer(created); + downstreamProducer = graph.getProducer(downstream); + viewProducer = graph.getProducer(view); + unboundedProducer = graph.getProducer(unbounded); } @Test @@ -146,7 +152,7 @@ public void writeToViewWriterThenReadReads() { @Test public void getExecutionContextSameStepSameKeyState() { DirectExecutionContext fooContext = - context.getExecutionContext(created.getProducingTransformInternal(), + context.getExecutionContext(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of())); StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); @@ -159,12 +165,12 @@ public void getExecutionContextSameStepSameKeyState() { .createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), created) .commit(Instant.now()), ImmutableList.of(), - StepTransformResult.withoutHold(created.getProducingTransformInternal()) + StepTransformResult.withoutHold(createdProducer) .withState(stepContext.commitState()) .build()); DirectExecutionContext secondFooContext = - context.getExecutionContext(created.getProducingTransformInternal(), + context.getExecutionContext(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of())); assertThat( secondFooContext @@ -179,7 +185,7 @@ public void getExecutionContextSameStepSameKeyState() { @Test public void getExecutionContextDifferentKeysIndependentState() { DirectExecutionContext fooContext = - context.getExecutionContext(created.getProducingTransformInternal(), + context.getExecutionContext(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of())); StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); @@ -191,7 +197,7 @@ public void getExecutionContextDifferentKeysIndependentState() { .add(1); DirectExecutionContext barContext = - context.getExecutionContext(created.getProducingTransformInternal(), + context.getExecutionContext(createdProducer, StructuralKey.of("bar", StringUtf8Coder.of())); assertThat(barContext, not(equalTo(fooContext))); assertThat( @@ -207,7 +213,7 @@ public void getExecutionContextDifferentKeysIndependentState() { public void getExecutionContextDifferentStepsIndependentState() { StructuralKey myKey = StructuralKey.of("foo", StringUtf8Coder.of()); DirectExecutionContext fooContext = - context.getExecutionContext(created.getProducingTransformInternal(), myKey); + context.getExecutionContext(createdProducer, myKey); StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); @@ -218,7 +224,7 @@ public void getExecutionContextDifferentStepsIndependentState() { .add(1); DirectExecutionContext barContext = - context.getExecutionContext(downstream.getProducingTransformInternal(), myKey); + context.getExecutionContext(downstreamProducer, myKey); assertThat( barContext .getOrCreateStepContext("s1", "s1") @@ -232,15 +238,15 @@ public void getExecutionContextDifferentStepsIndependentState() { public void handleResultCommitsAggregators() { Class fn = getClass(); DirectExecutionContext fooContext = - context.getExecutionContext(created.getProducingTransformInternal(), null); + context.getExecutionContext(createdProducer, null); DirectExecutionContext.StepContext stepContext = fooContext.createStepContext( - "STEP", created.getProducingTransformInternal().getTransform().getName()); + "STEP", createdProducer.getTransform().getName()); AggregatorContainer container = context.getAggregatorContainer(); AggregatorContainer.Mutator mutator = container.createMutator(); mutator.createAggregatorForDoFn(fn, stepContext, "foo", new SumLongFn()).addValue(4L); TransformResult result = - StepTransformResult.withoutHold(created.getProducingTransformInternal()) + StepTransformResult.withoutHold(createdProducer) .withAggregatorChanges(mutator) .build(); context.handleResult(null, ImmutableList.of(), result); @@ -250,7 +256,7 @@ public void handleResultCommitsAggregators() { mutatorAgain.createAggregatorForDoFn(fn, stepContext, "foo", new SumLongFn()).addValue(12L); TransformResult secondResult = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()) + StepTransformResult.withoutHold(downstreamProducer) .withAggregatorChanges(mutatorAgain) .build(); context.handleResult( @@ -264,7 +270,7 @@ public void handleResultCommitsAggregators() { public void handleResultStoresState() { StructuralKey myKey = StructuralKey.of("foo".getBytes(), ByteArrayCoder.of()); DirectExecutionContext fooContext = - context.getExecutionContext(downstream.getProducingTransformInternal(), myKey); + context.getExecutionContext(downstreamProducer, myKey); StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); @@ -276,7 +282,7 @@ public void handleResultStoresState() { bag.add(4); TransformResult stateResult = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()) + StepTransformResult.withoutHold(downstreamProducer) .withState(state) .build(); @@ -286,7 +292,7 @@ public void handleResultStoresState() { stateResult); DirectExecutionContext afterResultContext = - context.getExecutionContext(downstream.getProducingTransformInternal(), myKey); + context.getExecutionContext(downstreamProducer, myKey); CopyOnAccessInMemoryStateInternals afterResultState = afterResultContext.getOrCreateStepContext("s1", "s1").stateInternals(); @@ -309,7 +315,7 @@ public void run() { downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback); TransformResult result = - StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0)) + StepTransformResult.withHold(createdProducer, new Instant(0)) .build(); context.handleResult(null, ImmutableList.of(), result); @@ -318,7 +324,7 @@ public void run() { assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false)); TransformResult finishedResult = - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(createdProducer).build(); context.handleResult(null, ImmutableList.of(), finishedResult); context.forceRefresh(); // Obtain the value via blocking call @@ -328,7 +334,7 @@ public void run() { @Test public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception { TransformResult finishedResult = - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(createdProducer).build(); context.handleResult(null, ImmutableList.of(), finishedResult); final CountDownLatch callLatch = new CountDownLatch(1); @@ -348,7 +354,7 @@ public void run() { @Test public void extractFiredTimersExtractsTimers() { TransformResult holdResult = - StepTransformResult.withHold(created.getProducingTransformInternal(), new Instant(0)) + StepTransformResult.withHold(createdProducer, new Instant(0)) .build(); context.handleResult(null, ImmutableList.of(), holdResult); @@ -356,7 +362,7 @@ public void extractFiredTimersExtractsTimers() { TimerData toFire = TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME); TransformResult timerResult = - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()) + StepTransformResult.withoutHold(downstreamProducer) .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null)) .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build()) .build(); @@ -372,7 +378,7 @@ public void extractFiredTimersExtractsTimers() { assertThat(context.extractFiredTimers(), emptyIterable()); TransformResult advanceResult = - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(createdProducer).build(); // Should cause the downstream timer to fire context.handleResult(null, ImmutableList.of(), advanceResult); @@ -403,14 +409,14 @@ public void createKeyedBundleKeyed() { @Test public void isDoneWithUnboundedPCollectionAndShutdown() { context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(true); - assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false)); + assertThat(context.isDone(unboundedProducer), is(false)); context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(unboundedProducer).build()); context.extractFiredTimers(); - assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(true)); + assertThat(context.isDone(unboundedProducer), is(true)); } @Test @@ -428,14 +434,14 @@ public void isDoneWithUnboundedPCollectionAndNotShutdown() { @Test public void isDoneWithOnlyBoundedPCollections() { context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false); - assertThat(context.isDone(created.getProducingTransformInternal()), is(false)); + assertThat(context.isDone(createdProducer), is(false)); context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(createdProducer).build()); context.extractFiredTimers(); - assertThat(context.isDone(created.getProducingTransformInternal()), is(true)); + assertThat(context.isDone(createdProducer), is(true)); } @Test @@ -449,7 +455,7 @@ public void isDoneWithPartiallyDone() { context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(created.getProducingTransformInternal()) + StepTransformResult.withoutHold(createdProducer) .addOutput(rootBundle) .build()); @SuppressWarnings("unchecked") @@ -458,7 +464,7 @@ public void isDoneWithPartiallyDone() { context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(unboundedProducer).build()); assertThat(context.isDone(), is(false)); for (AppliedPTransform consumers : graph.getPrimitiveConsumers(created)) { @@ -479,22 +485,22 @@ public void isDoneWithUnboundedAndNotShutdown() { context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(createdProducer).build()); context.handleResult( null, ImmutableList.of(), - StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(unboundedProducer).build()); context.handleResult( context.createBundle(created).commit(Instant.now()), ImmutableList.of(), - StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(downstreamProducer).build()); context.extractFiredTimers(); assertThat(context.isDone(), is(false)); context.handleResult( context.createBundle(created).commit(Instant.now()), ImmutableList.of(), - StepTransformResult.withoutHold(view.getProducingTransformInternal()).build()); + StepTransformResult.withoutHold(viewProducer).build()); context.extractFiredTimers(); assertThat(context.isDone(), is(false)); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index cb27fbc923b9..9e22c362802b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -47,6 +47,7 @@ @RunWith(JUnit4.class) public class FlattenEvaluatorFactoryTest { private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + @Test public void testFlattenInMemoryEvaluator() throws Exception { TestPipeline p = TestPipeline.create(); @@ -69,10 +70,11 @@ public void testFlattenInMemoryEvaluator() throws Exception { when(context.createBundle(flattened)).thenReturn(flattenedLeftBundle, flattenedRightBundle); FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory(context); + AppliedPTransform flattenedProducer = DirectGraphs.getProducer(flattened); TransformEvaluator leftSideEvaluator = - factory.forApplication(flattened.getProducingTransformInternal(), leftBundle); + factory.forApplication(flattenedProducer, leftBundle); TransformEvaluator rightSideEvaluator = - factory.forApplication(flattened.getProducingTransformInternal(), rightBundle); + factory.forApplication(flattenedProducer, rightBundle); leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1)); rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1)); @@ -92,13 +94,13 @@ public void testFlattenInMemoryEvaluator() throws Exception { Matchers.>contains(flattenedRightBundle)); assertThat( rightSideResult.getTransform(), - Matchers.>equalTo(flattened.getProducingTransformInternal())); + Matchers.>equalTo(flattenedProducer)); assertThat( leftSideResult.getOutputBundles(), Matchers.>contains(flattenedLeftBundle)); assertThat( leftSideResult.getTransform(), - Matchers.>equalTo(flattened.getProducingTransformInternal())); + Matchers.>equalTo(flattenedProducer)); assertThat( flattenedLeftBundle.commit(Instant.now()).getElements(), @@ -126,9 +128,10 @@ public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Except .thenReturn(bundleFactory.createBundle(flattened)); FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory(evaluationContext); + AppliedPTransform flattendProducer = DirectGraphs.getProducer(flattened); TransformEvaluator emptyEvaluator = factory.forApplication( - flattened.getProducingTransformInternal(), + flattendProducer, bundleFactory.createRootBundle().commit(BoundedWindow.TIMESTAMP_MAX_VALUE)); TransformResult leftSideResult = emptyEvaluator.finishBundle(); @@ -138,7 +141,7 @@ public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Except assertThat(outputBundle.getElements(), emptyIterable()); assertThat( leftSideResult.getTransform(), - Matchers.>equalTo(flattened.getProducingTransformInternal())); + Matchers.>equalTo(flattendProducer)); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java index 7ba38ce728f5..f0b29f07ab62 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java @@ -97,7 +97,7 @@ public void testInMemoryEvaluator() throws Exception { ((KvCoder) values.getCoder()).getKeyCoder(); TransformEvaluator> evaluator = new GroupByKeyOnlyEvaluatorFactory(evaluationContext) - .forApplication(groupedKvs.getProducingTransformInternal(), inputBundle); + .forApplication(DirectGraphs.getProducer(groupedKvs), inputBundle); evaluator.processElement(WindowedValue.valueInGlobalWindow(firstFoo)); evaluator.processElement(WindowedValue.valueInGlobalWindow(secondFoo)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java index 23340c6d361a..7efdb3d2b875 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java @@ -90,8 +90,7 @@ public void testInMemoryEvaluator() throws Exception { ((KvCoder) values.getCoder()).getKeyCoder(); TransformEvaluator> evaluator = new GroupByKeyOnlyEvaluatorFactory(evaluationContext) - .forApplication( - groupedKvs.getProducingTransformInternal(), inputBundle); + .forApplication(DirectGraphs.getProducer(groupedKvs), inputBundle); evaluator.processElement(WindowedValue.valueInGlobalWindow(firstFoo)); evaluator.processElement(WindowedValue.valueInGlobalWindow(secondFoo)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index a65cd30b0a53..1ad6ba6af30c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -64,7 +64,7 @@ public void processElement(ProcessContext c) c.element()[0] = 'b'; } })); - consumer = pcollection.apply(Count.globally()).getProducingTransformInternal(); + consumer = DirectGraphs.getProducer(pcollection.apply(Count.globally())); } @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 85e99c51a2b4..d48ac142ad6d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -152,8 +152,9 @@ private ParDoEvaluator createEvaluator( when(evaluationContext.getAggregatorContainer()).thenReturn(container); when(evaluationContext.getAggregatorMutator()).thenReturn(mutator); + @SuppressWarnings("unchecked") AppliedPTransform, ?, ?> transform = - (AppliedPTransform, ?, ?>) output.getProducingTransformInternal(); + (AppliedPTransform, ?, ?>) DirectGraphs.getProducer(output); return ParDoEvaluator.create( evaluationContext, stepContext, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index ecf11edea3eb..06c85ef06914 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -129,7 +129,7 @@ public void process(ProcessContext c) {} AppliedPTransform< PCollection>>, PCollectionTuple, StatefulParDo> - producingTransform = (AppliedPTransform) produced.getProducingTransformInternal(); + producingTransform = (AppliedPTransform) DirectGraphs.getProducer(produced); // Then there will be a digging down to the step context to get the state internals when(mockEvaluationContext.getExecutionContext( @@ -239,7 +239,7 @@ public void process(ProcessContext c) {} AppliedPTransform< PCollection>>, PCollectionTuple, StatefulParDo> - producingTransform = (AppliedPTransform) produced.getProducingTransformInternal(); + producingTransform = (AppliedPTransform) DirectGraphs.getProducer(produced); // Then there will be a digging down to the step context to get the state internals when(mockEvaluationContext.getExecutionContext( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java index a21d8f7c9d0a..d3a2cca84444 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java @@ -48,7 +48,7 @@ public class StepTransformResultTest { public void setup() { TestPipeline p = TestPipeline.create(); pc = p.apply(Create.of(1, 2, 3)); - transform = pc.getProducingTransformInternal(); + transform = DirectGraphs.getGraph(p).getProducer(pc); bundleFactory = ImmutableListBundleFactory.create(); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index 3d31df6d2bb6..6bb86233a0dc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; @@ -80,15 +81,16 @@ public void producesElementsInSequence() throws Exception { when(context.createBundle(streamVals)) .thenReturn(bundleFactory.createBundle(streamVals), bundleFactory.createBundle(streamVals)); + AppliedPTransform streamProducer = DirectGraphs.getProducer(streamVals); Collection> initialInputs = new TestStreamEvaluatorFactory.InputProvider(context) - .getInitialInputs(streamVals.getProducingTransformInternal(), 1); + .getInitialInputs(streamProducer, 1); @SuppressWarnings("unchecked") CommittedBundle> initialBundle = (CommittedBundle>) Iterables.getOnlyElement(initialInputs); TransformEvaluator> firstEvaluator = - factory.forApplication(streamVals.getProducingTransformInternal(), initialBundle); + factory.forApplication(streamProducer, initialBundle); firstEvaluator.processElement(Iterables.getOnlyElement(initialBundle.getElements())); TransformResult> firstResult = firstEvaluator.finishBundle(); @@ -101,7 +103,7 @@ public void producesElementsInSequence() throws Exception { CommittedBundle> secondBundle = initialBundle.withElements(Collections.singleton(firstResidual)); TransformEvaluator> secondEvaluator = - factory.forApplication(streamVals.getProducingTransformInternal(), secondBundle); + factory.forApplication(streamProducer, secondBundle); secondEvaluator.processElement(firstResidual); TransformResult> secondResult = secondEvaluator.finishBundle(); @@ -114,7 +116,7 @@ public void producesElementsInSequence() throws Exception { CommittedBundle> thirdBundle = secondBundle.withElements(Collections.singleton(secondResidual)); TransformEvaluator> thirdEvaluator = - factory.forApplication(streamVals.getProducingTransformInternal(), thirdBundle); + factory.forApplication(streamProducer, thirdBundle); thirdEvaluator.processElement(secondResidual); TransformResult> thirdResult = thirdEvaluator.finishBundle(); @@ -128,7 +130,7 @@ public void producesElementsInSequence() throws Exception { CommittedBundle> fourthBundle = thirdBundle.withElements(Collections.singleton(thirdResidual)); TransformEvaluator> fourthEvaluator = - factory.forApplication(streamVals.getProducingTransformInternal(), fourthBundle); + factory.forApplication(streamProducer, fourthBundle); fourthEvaluator.processElement(thirdResidual); TransformResult> fourthResult = fourthEvaluator.finishBundle(); @@ -142,7 +144,7 @@ public void producesElementsInSequence() throws Exception { CommittedBundle> fifthBundle = thirdBundle.withElements(Collections.singleton(fourthResidual)); TransformEvaluator> fifthEvaluator = - factory.forApplication(streamVals.getProducingTransformInternal(), fifthBundle); + factory.forApplication(streamProducer, fifthBundle); fifthEvaluator.processElement(fourthResidual); TransformResult> fifthResult = fifthEvaluator.finishBundle(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java index 08b1e188484a..4ad22bc3af49 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java @@ -89,8 +89,9 @@ public void setup() { created = p.apply(Create.of("foo", "spam", "third")); PCollection> downstream = created.apply(WithKeys.of(3)); - createdProducer = created.getProducingTransformInternal(); - downstreamProducer = downstream.getProducingTransformInternal(); + DirectGraph graph = DirectGraphs.getGraph(p); + createdProducer = graph.getProducer(created); + downstreamProducer = graph.getProducer(downstream); when(evaluationContext.getMetrics()).thenReturn(metrics); } @@ -317,7 +318,7 @@ public TransformResult finishBundle() throws Exception { @Test public void callWithEnforcementThrowsOnFinishPropagates() throws Exception { final TransformResult result = - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(createdProducer).build(); TransformEvaluator evaluator = new TransformEvaluator() { @@ -356,7 +357,7 @@ public TransformResult finishBundle() throws Exception { @Test public void callWithEnforcementThrowsOnElementPropagates() throws Exception { final TransformResult result = - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(); + StepTransformResult.withoutHold(createdProducer).build(); TransformEvaluator evaluator = new TransformEvaluator() { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index 5a10134fb87c..dd36a2f8c1f7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import static org.apache.beam.runners.direct.DirectGraphs.getProducer; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -90,6 +91,7 @@ public class UnboundedReadEvaluatorFactoryTest { private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); private UnboundedSource source; + private DirectGraph graph; @Before public void setup() { @@ -100,6 +102,7 @@ public void setup() { context = mock(EvaluationContext.class); factory = new UnboundedReadEvaluatorFactory(context); output = bundleFactory.createBundle(longs); + graph = DirectGraphs.getGraph(p); when(context.createBundle(longs)).thenReturn(output); } @@ -115,7 +118,7 @@ public UncommittedBundle answer(InvocationOnMock invocation) throws Throwable int numSplits = 5; Collection> initialInputs = new UnboundedReadEvaluatorFactory.InputProvider(context) - .getInitialInputs(longs.getProducingTransformInternal(), numSplits); + .getInitialInputs(graph.getProducer(longs), numSplits); // CountingSource.unbounded has very good splitting behavior assertThat(initialInputs, hasSize(numSplits)); @@ -148,15 +151,14 @@ public void unboundedSourceInMemoryTransformEvaluatorProducesElements() throws E Collection> initialInputs = new UnboundedReadEvaluatorFactory.InputProvider(context) - .getInitialInputs(longs.getProducingTransformInternal(), 1); + .getInitialInputs(graph.getProducer(longs), 1); CommittedBundle inputShards = Iterables.getOnlyElement(initialInputs); UnboundedSourceShard inputShard = (UnboundedSourceShard) Iterables.getOnlyElement(inputShards.getElements()).getValue(); TransformEvaluator> evaluator = - factory.forApplication( - longs.getProducingTransformInternal(), inputShards); + factory.forApplication(graph.getProducer(longs), inputShards); evaluator.processElement((WindowedValue) Iterables.getOnlyElement(inputShards.getElements())); TransformResult> result = evaluator.finishBundle(); @@ -190,7 +192,7 @@ public void unboundedSourceWithDuplicatesMultipleCalls() throws Exception { TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); - AppliedPTransform sourceTransform = pcollection.getProducingTransformInternal(); + AppliedPTransform sourceTransform = getProducer(pcollection); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); Collection> initialInputs = @@ -233,7 +235,7 @@ public void noElementsAvailableReaderIncludedInResidual() throws Exception { // Read with a very slow rate so by the second read there are no more elements PCollection pcollection = p.apply(Read.from(new TestUnboundedSource<>(VarLongCoder.of(), 1L))); - AppliedPTransform sourceTransform = pcollection.getProducingTransformInternal(); + AppliedPTransform sourceTransform = DirectGraphs.getProducer(pcollection); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); Collection> initialInputs = @@ -291,7 +293,9 @@ public void evaluatorReusesReader() throws Exception { TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); - AppliedPTransform sourceTransform = pcollection.getProducingTransformInternal(); + DirectGraph graph = DirectGraphs.getGraph(p); + AppliedPTransform sourceTransform = + graph.getProducer(pcollection); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); UncommittedBundle output = bundleFactory.createBundle(pcollection); @@ -307,8 +311,7 @@ public void evaluatorReusesReader() throws Exception { .commit(Instant.now()); UnboundedReadEvaluatorFactory factory = new UnboundedReadEvaluatorFactory(context, 1.0 /* Always reuse */); - new UnboundedReadEvaluatorFactory.InputProvider(context) - .getInitialInputs(pcollection.getProducingTransformInternal(), 1); + new UnboundedReadEvaluatorFactory.InputProvider(context).getInitialInputs(sourceTransform, 1); TransformEvaluator> evaluator = factory.forApplication(sourceTransform, inputBundle); evaluator.processElement(shard); @@ -336,7 +339,8 @@ public void evaluatorClosesReaderAndResumesFromCheckpoint() throws Exception { TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); - AppliedPTransform sourceTransform = pcollection.getProducingTransformInternal(); + AppliedPTransform sourceTransform = + DirectGraphs.getGraph(p).getProducer(pcollection); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); UncommittedBundle output = bundleFactory.createBundle(pcollection); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index 7d140207497e..7c080099675a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.Values; @@ -73,9 +74,10 @@ public void testInMemoryEvaluator() throws Exception { CommittedBundle inputBundle = bundleFactory.createBundle(input).commit(Instant.now()); + AppliedPTransform producer = DirectGraphs.getProducer(view); TransformEvaluator> evaluator = new ViewEvaluatorFactory(context) - .forApplication(view.getProducingTransformInternal(), inputBundle); + .forApplication(producer, inputBundle); evaluator.processElement( WindowedValue.>valueInGlobalWindow(ImmutableList.of("foo", "bar"))); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java index 1be9a9884a8c..acdabb6b26bd 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java @@ -55,8 +55,10 @@ public class WatermarkCallbackExecutorTest { public void setup() { TestPipeline p = TestPipeline.create(); PCollection created = p.apply(Create.of(1, 2, 3)); - create = created.getProducingTransformInternal(); - sum = created.apply(Sum.integersGlobally()).getProducingTransformInternal(); + PCollection summed = created.apply(Sum.integersGlobally()); + DirectGraph graph = DirectGraphs.getGraph(p); + create = graph.getProducer(created); + sum = graph.getProducer(summed); } @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 076e0fb1643a..eb4d0cdc9966 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -31,7 +31,6 @@ import java.util.Collection; import java.util.Collections; import java.util.EnumSet; -import java.util.HashMap; import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; @@ -63,7 +62,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -116,33 +114,8 @@ public void processElement(ProcessContext c) throws Exception { PCollectionList preFlatten = PCollectionList.of(createdInts).and(intsToFlatten); flattened = preFlatten.apply("flattened", Flatten.pCollections()); - Collection> rootTransforms = - ImmutableList.>of( - createdInts.getProducingTransformInternal(), - intsToFlatten.getProducingTransformInternal()); - - Map>> consumers = new HashMap<>(); - consumers.put( - createdInts, - ImmutableList.>of(filtered.getProducingTransformInternal(), - keyed.getProducingTransformInternal(), flattened.getProducingTransformInternal())); - consumers.put( - filtered, - Collections.>singleton( - filteredTimesTwo.getProducingTransformInternal())); - consumers.put(filteredTimesTwo, Collections.>emptyList()); - consumers.put(keyed, Collections.>emptyList()); - - consumers.put( - intsToFlatten, - Collections.>singleton( - flattened.getProducingTransformInternal())); - consumers.put(flattened, Collections.>emptyList()); - clock = MockClock.fromInstant(new Instant(1000)); - DirectGraphVisitor visitor = new DirectGraphVisitor(); - p.traverseTopologically(visitor); - graph = visitor.getGraph(); + graph = DirectGraphs.getGraph(p); manager = WatermarkManager.create(clock, graph); bundleFactory = ImmutableListBundleFactory.create(); @@ -155,7 +128,7 @@ public void processElement(ProcessContext c) throws Exception { @Test public void getWatermarkForUntouchedTransform() { TransformWatermarks watermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(watermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); assertThat(watermarks.getOutputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); @@ -170,13 +143,13 @@ public void getWatermarkForUpdatedSourceTransform() { CommittedBundle output = multiWindowedBundle(createdInts, 1); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(output)), new Instant(8000L)); manager.refreshAll(); TransformWatermarks updatedSourceWatermark = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(new Instant(8000L))); } @@ -191,7 +164,7 @@ public void getWatermarkForMultiInputTransform() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(intsToFlatten.getProducingTransformInternal(), + result(graph.getProducer(intsToFlatten), null, Collections.>singleton(secondPcollectionBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -199,7 +172,7 @@ public void getWatermarkForMultiInputTransform() { // We didn't do anything for the first source, so we shouldn't have progressed the watermark TransformWatermarks firstSourceWatermark = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat( firstSourceWatermark.getOutputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE))); @@ -207,14 +180,14 @@ public void getWatermarkForMultiInputTransform() { // the Second Source output all of the elements so it should be done (with a watermark at the // end of time). TransformWatermarks secondSourceWatermark = - manager.getWatermarks(intsToFlatten.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(intsToFlatten)); assertThat( secondSourceWatermark.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); // We haven't consumed anything yet, so our watermark should be at the beginning of time TransformWatermarks transformWatermark = - manager.getWatermarks(flattened.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(flattened)); assertThat( transformWatermark.getInputWatermark(), not(laterThan(BoundedWindow.TIMESTAMP_MIN_VALUE))); assertThat( @@ -225,15 +198,15 @@ public void getWatermarkForMultiInputTransform() { // anything from the first PCollection yet; so our watermark shouldn't advance manager.updateWatermarks(secondPcollectionBundle, TimerUpdate.empty(), - result(flattened.getProducingTransformInternal(), + result(graph.getProducer(flattened), secondPcollectionBundle.withElements(Collections.>emptyList()), Collections.>singleton(flattenedBundleSecondCreate)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks transformAfterProcessing = - manager.getWatermarks(flattened.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(flattened)); manager.updateWatermarks(secondPcollectionBundle, TimerUpdate.empty(), - result(flattened.getProducingTransformInternal(), + result(graph.getProducer(flattened), secondPcollectionBundle.withElements(Collections.>emptyList()), Collections.>singleton(flattenedBundleSecondCreate)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -252,13 +225,13 @@ public void getWatermarkForMultiInputTransform() { // past the end of the global window manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(firstPcollectionBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks firstSourceWatermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat( firstSourceWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); @@ -266,7 +239,7 @@ public void getWatermarkForMultiInputTransform() { // We still haven't consumed any of the first source's input, so the watermark should still not // progress TransformWatermarks flattenAfterSourcesProduced = - manager.getWatermarks(flattened.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(flattened)); assertThat( flattenAfterSourcesProduced.getInputWatermark(), not(laterThan(firstCollectionTimestamp))); assertThat( @@ -276,7 +249,7 @@ public void getWatermarkForMultiInputTransform() { // end of the global window), we should have a watermark equal to the min among buffered // elements TransformWatermarks withBufferedElements = - manager.getWatermarks(flattened.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(flattened)); assertThat(withBufferedElements.getInputWatermark(), equalTo(firstCollectionTimestamp)); assertThat(withBufferedElements.getOutputWatermark(), equalTo(firstCollectionTimestamp)); @@ -284,13 +257,13 @@ public void getWatermarkForMultiInputTransform() { bundleFactory.createBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); manager.updateWatermarks(firstPcollectionBundle, TimerUpdate.empty(), - result(flattened.getProducingTransformInternal(), + result(graph.getProducer(flattened), firstPcollectionBundle.withElements(Collections.>emptyList()), Collections.>singleton(completedFlattenBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks afterConsumingAllInput = - manager.getWatermarks(flattened.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(flattened)); assertThat( afterConsumingAllInput.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); @@ -329,8 +302,8 @@ public void getWatermarkMultiIdenticalInput() { Map, Collection>> initialInputs = ImmutableMap., Collection>>builder() - .put( - created.getProducingTransformInternal(), + .put(graph.getProducer( + created), Collections.>singleton(root)) .build(); tstMgr.initialize(initialInputs); @@ -338,7 +311,7 @@ public void getWatermarkMultiIdenticalInput() { root, TimerUpdate.empty(), CommittedResult.create( - StepTransformResult.withoutHold(created.getProducingTransformInternal()).build(), + StepTransformResult.withoutHold(graph.getProducer(created)).build(), root.withElements(Collections.>emptyList()), Collections.singleton(createBundle), EnumSet.allOf(OutputType.class)), @@ -385,13 +358,13 @@ public void getWatermarkForMultiConsumedCollection() { TimestampedValue.of(3, new Instant(-1000L))); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), new Instant(Long.MAX_VALUE)); manager.refreshAll(); TransformWatermarks createdAfterProducing = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat( createdAfterProducing.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); @@ -402,20 +375,20 @@ public void getWatermarkForMultiConsumedCollection() { TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(keyBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks keyedWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat( keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat( keyedWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); TransformWatermarks filteredWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat(filteredWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L)))); assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L)))); @@ -423,13 +396,13 @@ public void getWatermarkForMultiConsumedCollection() { timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L))); manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks filteredProcessedWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat( filteredProcessedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); @@ -450,7 +423,7 @@ public void updateWatermarkWithWatermarkHolds() { TimestampedValue.of(3, new Instant(-1000L))); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), new Instant(Long.MAX_VALUE)); @@ -461,13 +434,13 @@ public void updateWatermarkWithWatermarkHolds() { TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(keyBundle)), new Instant(500L)); manager.refreshAll(); TransformWatermarks keyedWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat( keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L)))); @@ -494,27 +467,27 @@ public void updateWatermarkWithKeyedWatermarkHolds() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, ImmutableList.of(firstKeyBundle, secondKeyBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.updateWatermarks(firstKeyBundle, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), firstKeyBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), new Instant(-1000L)); manager.updateWatermarks(secondKeyBundle, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), secondKeyBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), new Instant(1234L)); manager.refreshAll(); TransformWatermarks filteredWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat(filteredWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L)))); @@ -524,7 +497,7 @@ public void updateWatermarkWithKeyedWatermarkHolds() { createdInts).commit(clock.now()); manager.updateWatermarks(fauxFirstKeyTimerBundle, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), fauxFirstKeyTimerBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -537,7 +510,7 @@ public void updateWatermarkWithKeyedWatermarkHolds() { createdInts).commit(clock.now()); manager.updateWatermarks(fauxSecondKeyTimerBundle, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), fauxSecondKeyTimerBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), new Instant(5678L)); @@ -546,7 +519,7 @@ public void updateWatermarkWithKeyedWatermarkHolds() { manager.updateWatermarks(fauxSecondKeyTimerBundle, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), fauxSecondKeyTimerBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -564,26 +537,26 @@ public void updateOutputWatermarkShouldBeMonotonic() { CommittedBundle firstInput = bundleFactory.createBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(firstInput)), new Instant(0L)); manager.refreshAll(); TransformWatermarks firstWatermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L))); CommittedBundle secondInput = bundleFactory.createBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(secondInput)), new Instant(-250L)); manager.refreshAll(); TransformWatermarks secondWatermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L)))); } @@ -599,7 +572,7 @@ public void updateWatermarkWithHoldsShouldBeMonotonic() { TimestampedValue.of(3, new Instant(-1000L))); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), new Instant(Long.MAX_VALUE)); @@ -610,20 +583,20 @@ public void updateWatermarkWithHoldsShouldBeMonotonic() { TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(keyBundle)), new Instant(500L)); manager.refreshAll(); TransformWatermarks keyedWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat( keyedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat(keyedWatermarks.getOutputWatermark(), not(laterThan(new Instant(500L)))); Instant oldOutputWatermark = keyedWatermarks.getOutputWatermark(); TransformWatermarks updatedWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat( updatedWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); // We added a hold prior to the old watermark; we shouldn't progress (due to the earlier hold) @@ -646,7 +619,7 @@ public void updateWatermarkWithUnprocessedElements() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -655,12 +628,12 @@ public void updateWatermarkWithUnprocessedElements() { TimestampedValue.of(KV.of("MyKey", 1), BoundedWindow.TIMESTAMP_MIN_VALUE)); manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), createdBundle.withElements(ImmutableList.of(second, third)), Collections.>singleton(keyBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks keyedWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); // the unprocessed second and third are readded to pending assertThat( keyedWatermarks.getInputWatermark(), not(laterThan(new Instant(-1000L)))); @@ -681,7 +654,7 @@ public void updateWatermarkWithCompletedElementsNotPending() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -689,15 +662,15 @@ public void updateWatermarkWithCompletedElementsNotPending() { manager.updateWatermarks( neverCreatedBundle, TimerUpdate.empty(), - result( - filtered.getProducingTransformInternal(), + result(graph.getProducer( + filtered), neverCreatedBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks filteredWms = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat(filteredWms.getInputWatermark(), equalTo(new Instant(22L))); } @@ -712,7 +685,7 @@ public void updateWatermarkWithLateData() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), sourceWatermark); @@ -724,13 +697,13 @@ public void updateWatermarkWithLateData() { // Finish processing the on-time data. The watermarks should progress to be equal to the source manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(keyBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks onTimeWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark)); assertThat(onTimeWatermarks.getOutputWatermark(), equalTo(sourceWatermark)); @@ -740,19 +713,19 @@ public void updateWatermarkWithLateData() { // we don't advance the watermark past the current watermark until we've consumed the late data manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(lateDataBundle)), new Instant(2_000_000L)); manager.refreshAll(); TransformWatermarks bufferedLateWm = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L))); // The input watermark should be held to its previous value (not advanced due to late data; not // moved backwards in the presence of watermarks due to monotonicity). TransformWatermarks lateDataBufferedWatermark = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat(lateDataBufferedWatermark.getInputWatermark(), not(earlierThan(sourceWatermark))); assertThat(lateDataBufferedWatermark.getOutputWatermark(), not(earlierThan(sourceWatermark))); @@ -760,7 +733,7 @@ public void updateWatermarkWithLateData() { timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); manager.updateWatermarks(lateDataBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), lateDataBundle.withElements(Collections.>emptyList()), Collections.>singleton(lateKeyedBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -770,7 +743,7 @@ public void updateWatermarkWithLateData() { public void updateWatermarkWithDifferentWindowedValueInstances() { manager.updateWatermarks( null, - TimerUpdate.empty(), result(createdInts.getProducingTransformInternal(), null, + TimerUpdate.empty(), result(graph.getProducer(createdInts), null, Collections.>singleton( bundleFactory .createBundle(createdInts) @@ -783,13 +756,13 @@ public void updateWatermarkWithDifferentWindowedValueInstances() { .commit(Instant.now()); manager.updateWatermarks(createdBundle, TimerUpdate.empty(), - result(keyed.getProducingTransformInternal(), + result(graph.getProducer(keyed), createdBundle.withElements(Collections.>emptyList()), Collections.>emptyList()), null); manager.refreshAll(); TransformWatermarks onTimeWatermarks = - manager.getWatermarks(keyed.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(keyed)); assertThat(onTimeWatermarks.getInputWatermark(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); } @@ -802,20 +775,20 @@ public void getWatermarksAfterOnlyEmptyOutput() { CommittedBundle emptyCreateOutput = multiWindowedBundle(createdInts); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(emptyCreateOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks updatedSourceWatermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat( updatedSourceWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); TransformWatermarks finishedFilterWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat( finishedFilterWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); @@ -833,7 +806,7 @@ public void getWatermarksAfterHoldAndEmptyOutput() { CommittedBundle firstCreateOutput = multiWindowedBundle(createdInts, 1, 2); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(firstCreateOutput)), new Instant(12_000L)); @@ -841,33 +814,33 @@ public void getWatermarksAfterHoldAndEmptyOutput() { CommittedBundle firstFilterOutput = multiWindowedBundle(filtered); manager.updateWatermarks(firstCreateOutput, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), firstCreateOutput.withElements(Collections.>emptyList()), Collections.>singleton(firstFilterOutput)), new Instant(10_000L)); manager.refreshAll(); TransformWatermarks firstFilterWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat(firstFilterWatermarks.getInputWatermark(), not(earlierThan(new Instant(12_000L)))); assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L)))); CommittedBundle emptyCreateOutput = multiWindowedBundle(createdInts); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(emptyCreateOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks updatedSourceWatermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat( updatedSourceWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); TransformWatermarks finishedFilterWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat( finishedFilterWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); @@ -877,14 +850,14 @@ public void getWatermarksAfterHoldAndEmptyOutput() { @Test public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { TransformWatermarks watermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(clock.now())); assertThat( watermarks.getSynchronizedProcessingOutputTime(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); TransformWatermarks filteredWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); // Non-root processing watermarks don't progress until data has been processed assertThat( filteredWatermarks.getSynchronizedProcessingInputTime(), @@ -898,18 +871,18 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks createAfterUpdate = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now())); assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now())); TransformWatermarks filterAfterProduced = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat( filterAfterProduced.getSynchronizedProcessingInputTime(), not(laterThan(clock.now()))); assertThat( @@ -929,13 +902,13 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { bundleFactory.createBundle(intsToFlatten).commit(new Instant(1250L)); manager.updateWatermarks(createOutput, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), createOutput.withElements(Collections.>emptyList()), Collections.>singleton(filterOutputBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks filterAfterConsumed = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); assertThat( filterAfterConsumed.getSynchronizedProcessingInputTime(), not(laterThan(createAfterUpdate.getSynchronizedProcessingOutputTime()))); @@ -955,16 +928,16 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { CommittedBundle createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createdBundle)), new Instant(1248L)); manager.refreshAll(); TransformWatermarks filteredWms = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); TransformWatermarks filteredDoubledWms = - manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filteredTimesTwo)); Instant initialFilteredWm = filteredWms.getSynchronizedProcessingOutputTime(); Instant initialFilteredDoubledWm = filteredDoubledWms.getSynchronizedProcessingOutputTime(); @@ -978,7 +951,7 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { TimerUpdate.builder(key).setTimer(pastTimer).setTimer(futureTimer).build(); manager.updateWatermarks(createdBundle, timers, - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1014,7 +987,7 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { manager.updateWatermarks(filteredTimerBundle, TimerUpdate.builder(key) .withCompletedTimers(Collections.singleton(pastTimer)).build(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), filteredTimerBundle.withElements(Collections.>emptyList()), Collections.>singleton(filteredTimerResult)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1029,7 +1002,7 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { manager.updateWatermarks(filteredTimerResult, TimerUpdate.empty(), - result(filteredTimesTwo.getProducingTransformInternal(), + result(graph.getProducer(filteredTimesTwo), filteredTimerResult.withElements(Collections.>emptyList()), Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1050,11 +1023,11 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { Instant startTime = clock.now(); TransformWatermarks watermarks = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(watermarks.getSynchronizedProcessingInputTime(), equalTo(startTime)); TransformWatermarks filteredWatermarks = - manager.getWatermarks(filtered.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filtered)); // Non-root processing watermarks don't progress until data has been processed assertThat( filteredWatermarks.getSynchronizedProcessingInputTime(), @@ -1068,13 +1041,13 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks createAfterUpdate = - manager.getWatermarks(createdInts.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(createdInts)); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now()))); assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now()))); @@ -1083,7 +1056,7 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { bundleFactory.createBundle(createdInts).commit(new Instant(750L)); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(createSecondOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1097,7 +1070,7 @@ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers( CommittedBundle created = multiWindowedBundle(createdInts, 1, 2, 3); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(created)), new Instant(40_900L)); @@ -1111,14 +1084,14 @@ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers( TimerUpdate.builder(StructuralKey.of("key", StringUtf8Coder.of())) .setTimer(upstreamProcessingTimer) .build(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), created.withElements(Collections.>emptyList()), Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks downstreamWms = - manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filteredTimesTwo)); assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now())); clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1133,7 +1106,7 @@ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers( manager.updateWatermarks(otherCreated, TimerUpdate.builder(StructuralKey.of("key", StringUtf8Coder.of())) .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)).build(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), otherCreated.withElements(Collections.>emptyList()), Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1148,7 +1121,7 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { manager.updateWatermarks( null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>singleton(created)), new Instant(29_919_235L)); @@ -1160,14 +1133,14 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { manager.updateWatermarks( created, TimerUpdate.empty(), - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), created.withElements(Collections.>emptyList()), Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); TransformWatermarks downstreamWms = - manager.getWatermarks(filteredTimesTwo.getProducingTransformInternal()); + manager.getWatermarks(graph.getProducer(filteredTimesTwo)); assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(clock.now())); clock.set(BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -1184,7 +1157,7 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { CommittedBundle createdBundle = multiWindowedBundle(filtered); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.singleton(createdBundle)), new Instant(1500L)); @@ -1206,7 +1179,7 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { manager.updateWatermarks(createdBundle, update, - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); @@ -1220,7 +1193,7 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>emptyList()), new Instant(50_000L)); @@ -1242,7 +1215,7 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { CommittedBundle createdBundle = multiWindowedBundle(filtered); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.singleton(createdBundle)), new Instant(1500L)); @@ -1264,7 +1237,7 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { manager.updateWatermarks( createdBundle, update, - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); @@ -1278,7 +1251,7 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { clock.set(new Instant(50_000L)); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>emptyList()), new Instant(50_000L)); @@ -1301,7 +1274,7 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { CommittedBundle createdBundle = multiWindowedBundle(filtered); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.singleton(createdBundle)), new Instant(1500L)); @@ -1323,7 +1296,7 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { manager.updateWatermarks( createdBundle, update, - result(filtered.getProducingTransformInternal(), + result(graph.getProducer(filtered), createdBundle.withElements(Collections.>emptyList()), Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); @@ -1338,7 +1311,7 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { clock.set(new Instant(50_000L)); manager.updateWatermarks(null, TimerUpdate.empty(), - result(createdInts.getProducingTransformInternal(), + result(graph.getProducer(createdInts), null, Collections.>emptyList()), new Instant(50_000L)); From ec1eff387a711039801289c8f59c4240b1f1d007 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 2 Dec 2016 14:26:04 -0800 Subject: [PATCH 087/279] Remove getProducingTransformInternal from DirectGraphVisitorTest --- .../direct/DirectGraphVisitorTest.java | 70 +++++++++++++------ 1 file changed, 47 insertions(+), 23 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java index fb84de8a4ad3..5ad278b11a9f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java @@ -19,25 +19,34 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import com.google.common.collect.Iterables; import java.io.Serializable; +import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.io.CountingInput; +import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; 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.Flatten.FlattenPCollectionList; 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.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -48,7 +57,6 @@ /** * Tests for {@link DirectGraphVisitor}. */ -// TODO: Replace uses of getProducing @RunWith(JUnit4.class) public class DirectGraphVisitorTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -81,26 +89,36 @@ public void processElement(DoFn.ProcessContext c) @Test public void getRootTransformsContainsPBegins() { PCollection created = p.apply(Create.of("foo", "bar")); - PCollection counted = p.apply(CountingInput.upTo(1234L)); + PCollection counted = p.apply(Read.from(CountingSource.upTo(1234L))); PCollection unCounted = p.apply(CountingInput.unbounded()); p.traverseTopologically(visitor); + DirectGraph graph = visitor.getGraph(); + assertThat(graph.getRootTransforms(), hasSize(3)); + List> unapplied = new ArrayList<>(); assertThat( - visitor.getGraph().getRootTransforms(), + graph.getRootTransforms(), Matchers.>containsInAnyOrder( - created.getProducingTransformInternal(), - counted.getProducingTransformInternal(), - unCounted.getProducingTransformInternal())); + graph.getProducer(created), graph.getProducer(counted), graph.getProducer(unCounted))); + for (AppliedPTransform root : graph.getRootTransforms()) { + assertTrue(root.getInput() instanceof PBegin); + assertThat(root.getOutput(), Matchers.isOneOf(created, counted, unCounted)); + } } @Test public void getRootTransformsContainsEmptyFlatten() { - PCollection empty = - PCollectionList.empty(p).apply(Flatten.pCollections()); + FlattenPCollectionList flatten = Flatten.pCollections(); + PCollectionList emptyList = PCollectionList.empty(p); + PCollection empty = emptyList.apply(flatten); p.traverseTopologically(visitor); + DirectGraph graph = visitor.getGraph(); assertThat( - visitor.getGraph().getRootTransforms(), - Matchers.>containsInAnyOrder( - empty.getProducingTransformInternal())); + graph.getRootTransforms(), + Matchers.>containsInAnyOrder(graph.getProducer(empty))); + AppliedPTransform onlyRoot = Iterables.getOnlyElement(graph.getRootTransforms()); + assertThat(onlyRoot.getTransform(), Matchers.>equalTo(flatten)); + assertThat(onlyRoot.getInput(), Matchers.equalTo(emptyList)); + assertThat(onlyRoot.getOutput(), Matchers.equalTo(empty)); } @Test @@ -122,16 +140,20 @@ public void processElement(DoFn.ProcessContext c) p.traverseTopologically(visitor); + DirectGraph graph = visitor.getGraph(); + AppliedPTransform transformedProducer = + graph.getProducer(transformed); + AppliedPTransform flattenedProducer = + graph.getProducer(flattened); + assertThat( - visitor.getGraph().getPrimitiveConsumers(created), + graph.getPrimitiveConsumers(created), Matchers.>containsInAnyOrder( - transformed.getProducingTransformInternal(), - flattened.getProducingTransformInternal())); + transformedProducer, flattenedProducer)); assertThat( - visitor.getGraph().getPrimitiveConsumers(transformed), - Matchers.>containsInAnyOrder( - flattened.getProducingTransformInternal())); - assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable()); + graph.getPrimitiveConsumers(transformed), + Matchers.>containsInAnyOrder(flattenedProducer)); + assertThat(graph.getPrimitiveConsumers(flattened), emptyIterable()); } @Test @@ -143,12 +165,14 @@ public void getValueToConsumersWithDuplicateInputSucceeds() { p.traverseTopologically(visitor); + DirectGraph graph = visitor.getGraph(); + AppliedPTransform flattenedProducer = graph.getProducer(flattened); + assertThat( - visitor.getGraph().getPrimitiveConsumers(created), - Matchers.>containsInAnyOrder( - flattened.getProducingTransformInternal(), - flattened.getProducingTransformInternal())); - assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable()); + graph.getPrimitiveConsumers(created), + Matchers.>containsInAnyOrder(flattenedProducer, + flattenedProducer)); + assertThat(graph.getPrimitiveConsumers(flattened), emptyIterable()); } @Test From 6b005311d4aace7112c62a3c1494035bcaaab4b4 Mon Sep 17 00:00:00 2001 From: sammcveety Date: Fri, 11 Nov 2016 12:42:22 -0600 Subject: [PATCH 088/279] Update subnetwork documentation --- .../dataflow/options/DataflowPipelineWorkerPoolOptions.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index b6ec9c31f29c..ffb5a3a9f687 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -154,9 +154,9 @@ public String create(PipelineOptions options) { * workers. * *

    Default is up to the Dataflow service. Expected format is - * regions/REGION/subnetworks/SUBNETWORK. - * - *

    You may also need to specify network option. + * regions/REGION/subnetworks/SUBNETWORK or the fully qualified subnetwork name, beginning with + * https://..., e.g. https://www.googleapis.com/compute/alpha/projects/PROJECT/ + * regions/REGION/subnetworks/SUBNETWORK */ @Description("GCE subnetwork for launching workers. For more information, see the reference " + "documentation https://cloud.google.com/compute/docs/networking. " From f3edaeb4f07c66d5ef48afdf16ebd2226f67cefa Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sun, 4 Dec 2016 14:28:39 -0800 Subject: [PATCH 089/279] Improve BigQuery load error message --- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 0e0124604142..c00c19d126f1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2140,15 +2140,16 @@ private void load( .setProjectId(projectId) .setJobId(jobId); jobService.startLoadJob(jobRef, loadConfig); - Status jobStatus = - parseStatus(jobService.pollJob(jobRef, Bound.LOAD_JOB_POLL_MAX_RETRIES)); + Job job = jobService.pollJob(jobRef, Bound.LOAD_JOB_POLL_MAX_RETRIES); + Status jobStatus = parseStatus(job); switch (jobStatus) { case SUCCEEDED: return; case UNKNOWN: throw new RuntimeException("Failed to poll the load job status of job " + jobId); case FAILED: - LOG.info("BigQuery load job failed: {}", jobId); + LOG.info("BigQuery load job failed. Status: {} Details: {}", + jobId, job.getStatus()); continue; default: throw new IllegalStateException(String.format("Unexpected job status: %s of job %s", From 2b044f3f315655b863dbc7fd298f33c196fb8ef7 Mon Sep 17 00:00:00 2001 From: Sam Whittle Date: Thu, 10 Nov 2016 12:59:49 -0800 Subject: [PATCH 090/279] Improve ReduceFnRunner prefetching - add prefetch* methods for prefetching state matching existing methods - prefetch triggers in processElements - replace onTimer with batched onTimers method to allow prefetching across timers Additionally remove deprecated TimerCallback usage --- .../GroupAlsoByWindowViaWindowSetDoFn.java | 11 +- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 48 +- .../beam/runners/core/PaneInfoTracker.java | 4 + .../runners/core/ReduceFnContextFactory.java | 9 +- .../beam/runners/core/ReduceFnRunner.java | 493 +++++++++++------- .../beam/runners/core/WatermarkHold.java | 5 + .../triggers/TriggerStateMachineRunner.java | 14 +- .../beam/runners/core/ReduceFnTester.java | 77 ++- .../triggers/TriggerStateMachineTester.java | 17 +- .../GroupAlsoByWindowEvaluatorFactory.java | 5 +- 10 files changed, 440 insertions(+), 243 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index 208226933eca..14171b37a57a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; @@ -72,9 +71,9 @@ private GroupAlsoByWindowViaWindowSetDoFn( @Override public void processElement(ProcessContext c) throws Exception { - KeyedWorkItem element = c.element(); + KeyedWorkItem keyedWorkItem = c.element(); - K key = c.element().key(); + K key = keyedWorkItem.key(); TimerInternals timerInternals = c.windowingInternals().timerInternals(); StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); @@ -92,10 +91,8 @@ public void processElement(ProcessContext c) throws Exception { reduceFn, c.getPipelineOptions()); - reduceFnRunner.processElements(element.elementsIterable()); - for (TimerData timer : element.timersIterable()) { - reduceFnRunner.onTimer(timer); - } + reduceFnRunner.processElements(keyedWorkItem.elementsIterable()); + reduceFnRunner.onTimers(keyedWorkItem.timersIterable()); reduceFnRunner.persist(); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index b4b366cb61f4..918919170ec6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -18,17 +18,18 @@ package org.apache.beam.runners.core; import com.google.common.collect.Iterables; +import java.util.ArrayList; import java.util.List; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; -import org.apache.beam.sdk.util.state.TimerCallback; import org.joda.time.Instant; /** @@ -59,9 +60,8 @@ public void processElement(ProcessContext c) throws Exception { // timer manager from the context because it doesn't exist. So we create one and emulate the // watermark, knowing that we have all data and it is in timestamp order. InMemoryTimerInternals timerInternals = new InMemoryTimerInternals(); - timerInternals.advanceProcessingTime(TimerCallback.NO_OP, Instant.now()); - timerInternals.advanceSynchronizedProcessingTime( - TimerCallback.NO_OP, BoundedWindow.TIMESTAMP_MAX_VALUE); + timerInternals.advanceProcessingTime(Instant.now()); + timerInternals.advanceSynchronizedProcessingTime(Instant.now()); StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); ReduceFnRunner reduceFnRunner = @@ -85,22 +85,50 @@ public void processElement(ProcessContext c) throws Exception { reduceFnRunner.processElements(chunk); // Then, since elements are sorted by their timestamp, advance the input watermark - // to the first element, and fire any timers that may have been scheduled. - timerInternals.advanceInputWatermark(reduceFnRunner, chunk.iterator().next().getTimestamp()); + // to the first element. + timerInternals.advanceInputWatermark(chunk.iterator().next().getTimestamp()); + // Advance the processing times. + timerInternals.advanceProcessingTime(Instant.now()); + timerInternals.advanceSynchronizedProcessingTime(Instant.now()); - // Fire any processing timers that need to fire - timerInternals.advanceProcessingTime(reduceFnRunner, Instant.now()); + // Fire all the eligible timers. + fireEligibleTimers(timerInternals, reduceFnRunner); // Leave the output watermark undefined. Since there's no late data in batch mode // there's really no need to track it as we do for streaming. } // Finish any pending windows by advancing the input watermark to infinity. - timerInternals.advanceInputWatermark(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE); + timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); // Finally, advance the processing time to infinity to fire any timers. - timerInternals.advanceProcessingTime(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE); + timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); + timerInternals.advanceSynchronizedProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); + + fireEligibleTimers(timerInternals, reduceFnRunner); reduceFnRunner.persist(); } + + private void fireEligibleTimers(InMemoryTimerInternals timerInternals, + ReduceFnRunner reduceFnRunner) throws Exception { + List timers = new ArrayList<>(); + while (true) { + TimerInternals.TimerData timer; + while ((timer = timerInternals.removeNextEventTimer()) != null) { + timers.add(timer); + } + while ((timer = timerInternals.removeNextProcessingTimer()) != null) { + timers.add(timer); + } + while ((timer = timerInternals.removeNextSynchronizedProcessingTimer()) != null) { + timers.add(timer); + } + if (timers.isEmpty()) { + break; + } + reduceFnRunner.onTimers(timers); + timers.clear(); + } + } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 8140243b342b..69a4cfd5b9ef 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -54,6 +54,10 @@ public void clear(StateAccessor state) { state.access(PANE_INFO_TAG).clear(); } + public void prefetchPaneInfo(ReduceFn.Context context) { + context.state().access(PaneInfoTracker.PANE_INFO_TAG).readLater(); + } + /** * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane * info includes the timing for the pane, who's calculation is quite subtle. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 539126a8be0d..c5bda9b46e91 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -37,7 +37,6 @@ import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.MergingStateAccessor; -import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateContext; @@ -117,7 +116,7 @@ public ReduceFn.ProcessValueContext forValue( } public ReduceFn.OnTriggerContext forTrigger(W window, - ReadableState pane, StateStyle style, OnTriggerCallbacks callbacks) { + PaneInfo pane, StateStyle style, OnTriggerCallbacks callbacks) { return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks); } @@ -389,11 +388,11 @@ public Timers timers() { private class OnTriggerContextImpl extends ReduceFn.OnTriggerContext { private final StateAccessorImpl state; - private final ReadableState pane; + private final PaneInfo pane; private final OnTriggerCallbacks callbacks; private final TimersImpl timers; - private OnTriggerContextImpl(StateAccessorImpl state, ReadableState pane, + private OnTriggerContextImpl(StateAccessorImpl state, PaneInfo pane, OnTriggerCallbacks callbacks) { reduceFn.super(); this.state = state; @@ -424,7 +423,7 @@ public StateAccessor state() { @Override public PaneInfo paneInfo() { - return pane.read(); + return pane; } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index a686f46b63cb..6f7bbcf673f7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -21,12 +21,15 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableSet; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -58,10 +61,8 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; -import org.apache.beam.sdk.util.state.TimerCallback; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; @@ -91,7 +92,7 @@ * @param The output type that will be produced for each key. * @param The type of windows this operates on. */ -public class ReduceFnRunner implements TimerCallback { +public class ReduceFnRunner { /** * The {@link ReduceFnRunner} depends on most aspects of the {@link WindowingStrategy}. @@ -268,6 +269,32 @@ boolean hasNoActiveWindows() { return activeWindows.getActiveAndNewWindows().isEmpty(); } + private Set openWindows(Collection windows) { + Set result = new HashSet<>(); + for (W window : windows) { + ReduceFn.Context directContext = contextFactory.base( + window, StateStyle.DIRECT); + if (!triggerRunner.isClosed(directContext.state())) { + result.add(window); + } + } + return result; + } + + private Collection windowsThatShouldFire(Set windows) throws Exception { + Collection result = new ArrayList<>(); + // Filter out timers that didn't trigger. + for (W window : windows) { + ReduceFn.Context directContext = + contextFactory.base(window, StateStyle.DIRECT); + if (triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + result.add(window); + } + } + return result; + } + /** * Incorporate {@code values} into the underlying reduce function, and manage holds, timers, * triggers, and window merging. @@ -293,25 +320,54 @@ boolean hasNoActiveWindows() { * */ public void processElements(Iterable> values) throws Exception { + if (!values.iterator().hasNext()) { + return; + } + + // Determine all the windows for elements. + Set windows = collectWindows(values); // If an incoming element introduces a new window, attempt to merge it into an existing // window eagerly. - Map windowToMergeResult = collectAndMergeWindows(values); + Map windowToMergeResult = mergeWindows(windows); + if (!windowToMergeResult.isEmpty()) { + // Update windows by removing all windows that were merged away and adding + // the windows they were merged to. We add after completing all the + // removals to avoid removing a window that was also added. + List addedWindows = new ArrayList<>(windowToMergeResult.size()); + for (Map.Entry entry : windowToMergeResult.entrySet()) { + windows.remove(entry.getKey()); + addedWindows.add(entry.getValue()); + } + windows.addAll(addedWindows); + } - Set windowsToConsider = new HashSet<>(); + prefetchWindowsForValues(windows); - // Process each element, using the updated activeWindows determined by collectAndMergeWindows. + // All windows that are open before element processing may need to fire. + Set windowsToConsider = openWindows(windows); + + // Process each element, using the updated activeWindows determined by mergeWindows. for (WindowedValue value : values) { - windowsToConsider.addAll(processElement(windowToMergeResult, value)); + processElement(windowToMergeResult, value); } - // Trigger output from any window for which the trigger is ready + // Now that we've processed the elements, see if any of the windows need to fire. + // Prefetch state necessary to determine if the triggers should fire. for (W mergedWindow : windowsToConsider) { - ReduceFn.Context directContext = - contextFactory.base(mergedWindow, StateStyle.DIRECT); - ReduceFn.Context renamedContext = - contextFactory.base(mergedWindow, StateStyle.RENAMED); - triggerRunner.prefetchShouldFire(mergedWindow, directContext.state()); - emitIfAppropriate(directContext, renamedContext); + triggerRunner.prefetchShouldFire( + mergedWindow, contextFactory.base(mergedWindow, StateStyle.DIRECT).state()); + } + // Filter to windows that are firing. + Collection windowsToFire = windowsThatShouldFire(windowsToConsider); + // Prefetch windows that are firing. + for (W window : windowsToFire) { + prefetchEmit(contextFactory.base(window, StateStyle.DIRECT), + contextFactory.base(window, StateStyle.RENAMED)); + } + // Trigger output from firing windows. + for (W window : windowsToFire) { + emit(contextFactory.base(window, StateStyle.DIRECT), + contextFactory.base(window, StateStyle.RENAMED)); } // We're all done with merging and emitting elements so can compress the activeWindow state. @@ -325,52 +381,61 @@ public void persist() { } /** - * Extract the windows associated with the values, and invoke merge. Return a map - * from windows to the merge result window. If a window is not in the domain of - * the result map then it did not get merged into a different window. + * Extract the windows associated with the values. */ - private Map collectAndMergeWindows(Iterable> values) - throws Exception { - // No-op if no merging can take place + private Set collectWindows(Iterable> values) throws Exception { + Set windows = new HashSet<>(); + for (WindowedValue value : values) { + for (BoundedWindow untypedWindow : value.getWindows()) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + windows.add(window); + } + } + return windows; + } + + /** + * Invoke merge for the given windows and return a map from windows to the + * merge result window. Windows that were not merged are not present in the + * map. + */ + private Map mergeWindows(Set windows) throws Exception { if (windowingStrategy.getWindowFn().isNonMerging()) { - return ImmutableMap.of(); + // Return an empty map, indicating that every window is not merged. + return Collections.emptyMap(); } + Map windowToMergeResult = new HashMap<>(); // Collect the windows from all elements (except those which are too late) and // make sure they are already in the active window set or are added as NEW windows. - for (WindowedValue value : values) { - for (BoundedWindow untypedWindow : value.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - - // For backwards compat with pre 1.4 only. - // We may still have ACTIVE windows with multiple state addresses, representing - // a window who's state has not yet been eagerly merged. - // We'll go ahead and merge that state now so that we don't have to worry about - // this legacy case anywhere else. - if (activeWindows.isActive(window)) { - Set stateAddressWindows = activeWindows.readStateAddresses(window); - if (stateAddressWindows.size() > 1) { - // This is a legacy window who's state has not been eagerly merged. - // Do that now. - ReduceFn.OnMergeContext premergeContext = - contextFactory.forPremerge(window); - reduceFn.onMerge(premergeContext); - watermarkHold.onMerge(premergeContext); - activeWindows.merged(window); - } + for (W window : windows) { + // For backwards compat with pre 1.4 only. + // We may still have ACTIVE windows with multiple state addresses, representing + // a window who's state has not yet been eagerly merged. + // We'll go ahead and merge that state now so that we don't have to worry about + // this legacy case anywhere else. + if (activeWindows.isActive(window)) { + Set stateAddressWindows = activeWindows.readStateAddresses(window); + if (stateAddressWindows.size() > 1) { + // This is a legacy window who's state has not been eagerly merged. + // Do that now. + ReduceFn.OnMergeContext premergeContext = + contextFactory.forPremerge(window); + reduceFn.onMerge(premergeContext); + watermarkHold.onMerge(premergeContext); + activeWindows.merged(window); } - - // Add this window as NEW if it is not currently ACTIVE. - // If we had already seen this window and closed its trigger, then the - // window will not be currently ACTIVE. It will then be added as NEW here, - // and fall into the merging logic as usual. - activeWindows.ensureWindowExists(window); } + + // Add this window as NEW if it is not currently ACTIVE. + // If we had already seen this window and closed its trigger, then the + // window will not be currently ACTIVE. It will then be added as NEW here, + // and fall into the merging logic as usual. + activeWindows.ensureWindowExists(window); } // Merge all of the active windows and retain a mapping from source windows to result windows. - Map windowToMergeResult = new HashMap<>(); activeWindows.merge(new OnMergeCallback(windowToMergeResult)); return windowToMergeResult; } @@ -472,38 +537,50 @@ public void onMerge(Collection toBeMerged, W mergeResult) throws Exception { } /** - * Process an element. - * - * @param value the value being processed - * @return the set of windows in which the element was actually processed + * Redirect element windows to the ACTIVE windows they have been merged into. + * The compressed representation (value, {window1, window2, ...}) actually represents + * distinct elements (value, window1), (value, window2), ... + * so if window1 and window2 merge, the resulting window will contain both copies + * of the value. */ - private Collection processElement(Map windowToMergeResult, WindowedValue value) - throws Exception { - // Redirect element windows to the ACTIVE windows they have been merged into. - // The compressed representation (value, {window1, window2, ...}) actually represents - // distinct elements (value, window1), (value, window2), ... - // so if window1 and window2 merge, the resulting window will contain both copies - // of the value. - Collection windows = new ArrayList<>(); - for (BoundedWindow untypedWindow : value.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - W mergeResult = windowToMergeResult.get(window); - if (mergeResult == null) { - mergeResult = window; - } - windows.add(mergeResult); - } + private ImmutableSet toMergedWindows(final Map windowToMergeResult, + final Collection windows) { + return ImmutableSet.copyOf( + FluentIterable.from(windows).transform( + new Function() { + @Override + public W apply(BoundedWindow untypedWindow) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + W mergedWindow = windowToMergeResult.get(window); + // If the element is not present in the map, the window is unmerged. + return (mergedWindow == null) ? window : mergedWindow; + } + } + )); + } + private void prefetchWindowsForValues(Collection windows) { // Prefetch in each of the windows if we're going to need to process triggers for (W window : windows) { - ReduceFn.ProcessValueContext directContext = contextFactory.forValue( - window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT); + ReduceFn.Context directContext = contextFactory.base( + window, StateStyle.DIRECT); triggerRunner.prefetchForValue(window, directContext.state()); } + } + + /** + * Process an element. + * + * @param windowToMergeResult map of windows to merged windows. If a window is + * not present it is unmerged. + * @param value the value being processed + */ + private void processElement(Map windowToMergeResult, WindowedValue value) + throws Exception { + ImmutableSet windows = toMergedWindows(windowToMergeResult, value.getWindows()); // Process the element for each (mergeResultWindow, not closed) window it belongs to. - List triggerableWindows = new ArrayList<>(windows.size()); for (W window : windows) { ReduceFn.ProcessValueContext directContext = contextFactory.forValue( window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT); @@ -518,7 +595,6 @@ private Collection processElement(Map windowToMergeResult, WindowedValu continue; } - triggerableWindows.add(window); activeWindows.ensureWindowIsActive(window); ReduceFn.ProcessValueContext renamedContext = contextFactory.forValue( window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED); @@ -562,102 +638,152 @@ private Collection processElement(Map windowToMergeResult, WindowedValu // cannot take a trigger state from firing to non-firing. // (We don't actually assert this since it is too slow.) } - - return triggerableWindows; } /** - * Called when an end-of-window, garbage collection, or trigger-specific timer fires. + * Enriches TimerData with state necessary for processing a timer as well as + * common queries about a timer. */ - public void onTimer(TimerData timer) throws Exception { - // Which window is the timer for? - checkArgument(timer.getNamespace() instanceof WindowNamespace, - "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); - @SuppressWarnings("unchecked") - WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); - W window = windowNamespace.getWindow(); - ReduceFn.Context directContext = - contextFactory.base(window, StateStyle.DIRECT); - ReduceFn.Context renamedContext = - contextFactory.base(window, StateStyle.RENAMED); + private class EnrichedTimerData { + public final Instant timestamp; + public final ReduceFn.Context directContext; + public final ReduceFn.Context renamedContext; + // If this is an end-of-window timer then we may need to set a garbage collection timer + // if allowed lateness is non-zero. + public final boolean isEndOfWindow; + // If this is a garbage collection timer then we should trigger and + // garbage collect the window. We'll consider any timer at or after the + // end-of-window time to be a signal to garbage collect. + public final boolean isGarbageCollection; + + EnrichedTimerData( + TimerData timer, + ReduceFn.Context directContext, + ReduceFn.Context renamedContext) { + this.timestamp = timer.getTimestamp(); + this.directContext = directContext; + this.renamedContext = renamedContext; + W window = directContext.window(); + this.isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain() + && timer.getTimestamp().equals(window.maxTimestamp()); + Instant cleanupTime = garbageCollectionTime(window); + this.isGarbageCollection = !timer.getTimestamp().isBefore(cleanupTime); + } // Has this window had its trigger finish? // - The trigger may implement isClosed as constant false. // - If the window function does not support windowing then all windows will be considered // active. // So we must take conjunction of activeWindows and triggerRunner state. - boolean windowIsActiveAndOpen = - activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state()); + public boolean windowIsActiveAndOpen() { + return activeWindows.isActive(directContext.window()) + && !triggerRunner.isClosed(directContext.state()); + } + } - if (!windowIsActiveAndOpen) { - WindowTracing.debug( - "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window); + public void onTimers(Iterable timers) throws Exception { + if (!timers.iterator().hasNext()) { + return; } - // If this is an end-of-window timer then we may need to set a garbage collection timer - // if allowed lateness is non-zero. - boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain() - && timer.getTimestamp().equals(window.maxTimestamp()); - - // If this is a garbage collection timer then we should trigger and garbage collect the window. - // We'll consider any timer at or after the end-of-window time to be a signal to garbage - // collect. - Instant cleanupTime = garbageCollectionTime(window); - boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain() - && !timer.getTimestamp().isBefore(cleanupTime); - - if (isGarbageCollection) { - WindowTracing.debug( - "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - - if (windowIsActiveAndOpen) { - // We need to call onTrigger to emit the final pane if required. - // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted, - // and the watermark has passed the end of the window. - @Nullable Instant newHold = - onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow); - checkState(newHold == null, - "Hold placed at %s despite isFinished being true.", newHold); + // Create a reusable context for each timer and begin prefetching necessary + // state. + List enrichedTimers = new LinkedList(); + for (TimerData timer : timers) { + checkArgument(timer.getNamespace() instanceof WindowNamespace, + "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); + @SuppressWarnings("unchecked") + WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); + W window = windowNamespace.getWindow(); + ReduceFn.Context directContext = + contextFactory.base(window, StateStyle.DIRECT); + ReduceFn.Context renamedContext = + contextFactory.base(window, StateStyle.RENAMED); + EnrichedTimerData enrichedTimer = new EnrichedTimerData(timer, directContext, renamedContext); + enrichedTimers.add(enrichedTimer); + + // Perform prefetching of state to determine if the trigger should fire. + if (enrichedTimer.isGarbageCollection) { + triggerRunner.prefetchIsClosed(directContext.state()); + } else { + triggerRunner.prefetchShouldFire(directContext.window(), directContext.state()); } + } - // Cleanup flavor B: Clear all the remaining state for this window since we'll never - // see elements for it again. - clearAllState(directContext, renamedContext, windowIsActiveAndOpen); - } else { - WindowTracing.debug( - "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - if (windowIsActiveAndOpen) { - emitIfAppropriate(directContext, renamedContext); + // For those windows that are active and open, prefetch the triggering or emitting state. + for (EnrichedTimerData timer : enrichedTimers) { + if (timer.windowIsActiveAndOpen()) { + ReduceFn.Context directContext = timer.directContext; + if (timer.isGarbageCollection) { + prefetchOnTrigger(directContext, timer.renamedContext); + } else if (triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + prefetchEmit(directContext, timer.renamedContext); + } } + } - if (isEndOfWindow) { - // If the window strategy trigger includes a watermark trigger then at this point - // there should be no data holds, either because we'd already cleared them on an - // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate. - // We could assert this but it is very expensive. - - // Since we are processing an on-time firing we should schedule the garbage collection - // timer. (If getAllowedLateness is zero then the timer event will be considered a - // cleanup event and handled by the above). - // Note we must do this even if the trigger is finished so that we are sure to cleanup - // any final trigger finished bits. - checkState( - windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), - "Unexpected zero getAllowedLateness"); - WindowTracing.debug( - "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " - + "inputWatermark:{}; outputWatermark:{}", - key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), + // Perform processing now that everything is prefetched. + for (EnrichedTimerData timer : enrichedTimers) { + ReduceFn.Context directContext = timer.directContext; + ReduceFn.Context renamedContext = timer.renamedContext; + + if (timer.isGarbageCollection) { + WindowTracing.debug("ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), timer.timestamp, + timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); - checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), - "Cleanup time %s is beyond end-of-time", cleanupTime); - directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); + + boolean windowIsActiveAndOpen = timer.windowIsActiveAndOpen(); + if (windowIsActiveAndOpen) { + // We need to call onTrigger to emit the final pane if required. + // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted, + // and the watermark has passed the end of the window. + @Nullable + Instant newHold = onTrigger( + directContext, renamedContext, true /* isFinished */, timer.isEndOfWindow); + checkState(newHold == null, "Hold placed at %s despite isFinished being true.", newHold); + } + + // Cleanup flavor B: Clear all the remaining state for this window since we'll never + // see elements for it again. + clearAllState(directContext, renamedContext, windowIsActiveAndOpen); + } else { + WindowTracing.debug("ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), timer.timestamp, + timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + if (timer.windowIsActiveAndOpen() + && triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())) { + emit(directContext, renamedContext); + } + + if (timer.isEndOfWindow) { + // If the window strategy trigger includes a watermark trigger then at this point + // there should be no data holds, either because we'd already cleared them on an + // earlier onTrigger, or because we just cleared them on the above emit. + // We could assert this but it is very expensive. + + // Since we are processing an on-time firing we should schedule the garbage collection + // timer. (If getAllowedLateness is zero then the timer event will be considered a + // cleanup event and handled by the above). + // Note we must do this even if the trigger is finished so that we are sure to cleanup + // any final trigger finished bits. + checkState(windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), + "Unexpected zero getAllowedLateness"); + Instant cleanupTime = garbageCollectionTime(directContext.window()); + WindowTracing.debug( + "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Cleanup time %s is beyond end-of-time", cleanupTime); + directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); + } } } } @@ -666,7 +792,7 @@ public void onTimer(TimerData timer) throws Exception { * Clear all the state associated with {@code context}'s window. * Should only be invoked if we know all future elements for this window will be considered * beyond allowed lateness. - * This is a superset of the clearing done by {@link #emitIfAppropriate} below since: + * This is a superset of the clearing done by {@link #emit} below since: *

      *
    1. We can clear the trigger finished bits since we'll never need to ask if the trigger is * closed again. @@ -692,10 +818,10 @@ private void clearAllState( } else { // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2). // For (1), if !activeWindows.isActive then the window must be merging and has been - // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired + // explicitly removed by emit. But in that case the trigger must have fired // and been closed, so this case reduces to (2). // For (2), if triggerRunner.isClosed then the trigger was fired and entered the - // closed state. In that case emitIfAppropriate will have cleared all state in + // closed state. In that case emit will have cleared all state in // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows. // We also know nonEmptyPanes must have been unconditionally cleared by the trigger. // Since the trigger fired the existing watermark holds must have been cleared, and since @@ -737,17 +863,23 @@ private boolean shouldDiscardAfterFiring(boolean isFinished) { return false; } + private void prefetchEmit(ReduceFn.Context directContext, + ReduceFn.Context renamedContext) { + triggerRunner.prefetchShouldFire(directContext.window(), directContext.state()); + triggerRunner.prefetchOnFire(directContext.window(), directContext.state()); + triggerRunner.prefetchIsClosed(directContext.state()); + prefetchOnTrigger(directContext, renamedContext); + } + /** - * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state. + * Emit if a trigger is ready to fire or timers require it, and cleanup state. */ - private void emitIfAppropriate(ReduceFn.Context directContext, + private void emit( + ReduceFn.Context directContext, ReduceFn.Context renamedContext) throws Exception { - if (!triggerRunner.shouldFire( - directContext.window(), directContext.timers(), directContext.state())) { - // Ignore unless trigger is ready to fire - return; - } + checkState(triggerRunner.shouldFire( + directContext.window(), directContext.timers(), directContext.state())); // Inform the trigger of the transition to see if it is finished triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state()); @@ -782,7 +914,7 @@ private void emitIfAppropriate(ReduceFn.Context directCon } /** - * Do we need to emit a pane? + * Do we need to emit? */ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) { if (!isEmpty) { @@ -800,6 +932,15 @@ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing return false; } + private void prefetchOnTrigger( + final ReduceFn.Context directContext, + ReduceFn.Context renamedContext) { + paneInfoTracker.prefetchPaneInfo(directContext); + watermarkHold.prefetchExtract(renamedContext); + nonEmptyPanes.isEmpty(renamedContext.state()).readLater(); + reduceFn.prefetchOnTrigger(directContext.state()); + } + /** * Run the {@link ReduceFn#onTrigger} method and produce any necessary output. * @@ -813,25 +954,17 @@ private Instant onTrigger( throws Exception { Instant inputWM = timerInternals.currentInputWatermarkTime(); - // Prefetch necessary states - ReadableState outputTimestampFuture = - watermarkHold.extractAndRelease(renamedContext, isFinished).readLater(); - ReadableState paneFuture = - paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater(); - ReadableState isEmptyFuture = - nonEmptyPanes.isEmpty(renamedContext.state()).readLater(); - - reduceFn.prefetchOnTrigger(directContext.state()); - triggerRunner.prefetchOnFire(directContext.window(), directContext.state()); - // Calculate the pane info. - final PaneInfo pane = paneFuture.read(); - // Extract the window hold, and as a side effect clear it. + final PaneInfo pane = paneInfoTracker.getNextPaneInfo(directContext, isFinished).read(); - WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read(); + // Extract the window hold, and as a side effect clear it. + final WatermarkHold.OldAndNewHolds pair = + watermarkHold.extractAndRelease(renamedContext, isFinished).read(); final Instant outputTimestamp = pair.oldHold; @Nullable Instant newHold = pair.newHold; + final boolean isEmpty = nonEmptyPanes.isEmpty(renamedContext.state()).read(); + if (newHold != null) { // We can't be finished yet. checkState( @@ -863,11 +996,11 @@ private Instant onTrigger( } // Only emit a pane if it has data or empty panes are observable. - if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) { + if (needToEmit(isEmpty, isFinished, pane.getTiming())) { // Run reduceFn.onTrigger method. final List windows = Collections.singletonList(directContext.window()); ReduceFn.OnTriggerContext renamedTriggerContext = - contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED, + contextFactory.forTrigger(directContext.window(), pane, StateStyle.RENAMED, new OnTriggerCallbacks() { @Override public void output(OutputT toOutput) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index 3c0457178634..7f1afcc0a79c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -444,6 +444,11 @@ public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) { } } + public void prefetchExtract(final ReduceFn.Context context) { + context.state().access(elementHoldTag).readLater(); + context.state().access(EXTRA_HOLD_TAG).readLater(); + } + /** * Return (a future for) the earliest hold for {@code context}. Clear all the holds after * reading, but add/restore an end-of-window or garbage collection hold if required. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java index 9f03216f5b9b..2f277eb5bce3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java @@ -99,25 +99,25 @@ public boolean isClosed(StateAccessor state) { return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger); } - public void prefetchForValue(W window, StateAccessor state) { + public void prefetchIsClosed(StateAccessor state) { if (isFinishedSetNeeded()) { state.access(FINISHED_BITS_TAG).readLater(); } + } + + public void prefetchForValue(W window, StateAccessor state) { + prefetchIsClosed(state); rootTrigger.getSpec().prefetchOnElement( contextFactory.createStateAccessor(window, rootTrigger)); } public void prefetchOnFire(W window, StateAccessor state) { - if (isFinishedSetNeeded()) { - state.access(FINISHED_BITS_TAG).readLater(); - } + prefetchIsClosed(state); rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger)); } public void prefetchShouldFire(W window, StateAccessor state) { - if (isFinishedSetNeeded()) { - state.access(FINISHED_BITS_TAG).readLater(); - } + prefetchIsClosed(state); rootTrigger.getSpec().prefetchShouldFire( contextFactory.createStateAccessor(window, rootTrigger)); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 337be23610a2..db0cf9186a7d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -78,7 +78,6 @@ import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.TestInMemoryStateInternals; -import org.apache.beam.sdk.util.state.TimerCallback; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; @@ -100,7 +99,7 @@ public class ReduceFnTester { private final TestInMemoryStateInternals stateInternals = new TestInMemoryStateInternals<>(KEY); - private final TestTimerInternals timerInternals = new TestTimerInternals(); + private final InMemoryTimerInternals timerInternals = new InMemoryTimerInternals(); private final WindowFn windowFn; private final TestOutputWindowedValue testOutputter; @@ -443,8 +442,29 @@ public WindowedValue apply(WindowedValue> input) { * fire. Then advance the output watermark as far as possible. */ public void advanceInputWatermark(Instant newInputWatermark) throws Exception { + timerInternals.advanceInputWatermark(newInputWatermark); ReduceFnRunner runner = createRunner(); - timerInternals.advanceInputWatermark(runner, newInputWatermark); + while (true) { + TimerData timer; + List timers = new ArrayList<>(); + while ((timer = timerInternals.removeNextEventTimer()) != null) { + timers.add(timer); + } + if (timers.isEmpty()) { + break; + } + runner.onTimers(timers); + } + if (autoAdvanceOutputWatermark) { + Instant hold = stateInternals.earliestWatermarkHold(); + if (hold == null) { + WindowTracing.trace( + "TestInMemoryTimerInternals.advanceInputWatermark: no holds, " + + "so output watermark = input watermark"); + hold = timerInternals.currentInputWatermarkTime(); + } + advanceOutputWatermark(hold); + } runner.persist(); } @@ -458,8 +478,19 @@ public void advanceOutputWatermark(Instant newOutputWatermark) throws Exception /** Advance the processing time to the specified time, firing any timers that should fire. */ public void advanceProcessingTime(Instant newProcessingTime) throws Exception { + timerInternals.advanceProcessingTime(newProcessingTime); ReduceFnRunner runner = createRunner(); - timerInternals.advanceProcessingTime(runner, newProcessingTime); + while (true) { + TimerData timer; + List timers = new ArrayList<>(); + while ((timer = timerInternals.removeNextProcessingTimer()) != null) { + timers.add(timer); + } + if (timers.isEmpty()) { + break; + } + runner.onTimers(timers); + } runner.persist(); } @@ -467,9 +498,21 @@ public void advanceProcessingTime(Instant newProcessingTime) throws Exception { * Advance the synchronized processing time to the specified time, * firing any timers that should fire. */ - public void advanceSynchronizedProcessingTime(Instant newProcessingTime) throws Exception { + public void advanceSynchronizedProcessingTime( + Instant newSynchronizedProcessingTime) throws Exception { + timerInternals.advanceSynchronizedProcessingTime(newSynchronizedProcessingTime); ReduceFnRunner runner = createRunner(); - timerInternals.advanceSynchronizedProcessingTime(runner, newProcessingTime); + while (true) { + TimerData timer; + List timers = new ArrayList<>(); + while ((timer = timerInternals.removeNextSynchronizedProcessingTimer()) != null) { + timers.add(timer); + } + if (timers.isEmpty()) { + break; + } + runner.onTimers(timers); + } runner.persist(); } @@ -509,8 +552,10 @@ public WindowedValue apply(TimestampedValue input) { public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception { ReduceFnRunner runner = createRunner(); - runner.onTimer( + ArrayList timers = new ArrayList(1); + timers.add( TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain)); + runner.onTimers(timers); runner.persist(); } @@ -601,22 +646,4 @@ public long getSum() { return sum; } } - - private class TestTimerInternals extends InMemoryTimerInternals { - @Override - public void advanceInputWatermark(TimerCallback timerCallback, Instant newInputWatermark) - throws Exception { - super.advanceInputWatermark(timerCallback, newInputWatermark); - if (autoAdvanceOutputWatermark) { - Instant hold = stateInternals.earliestWatermarkHold(); - if (hold == null) { - WindowTracing.trace( - "TestInMemoryTimerInternals.advanceInputWatermark: no holds, " - + "so output watermark = input watermark"); - hold = currentInputWatermarkTime(); - } - advanceOutputWatermark(hold); - } - } - } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 1ccca170bd76..ed5ce9c960fb 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -53,7 +53,6 @@ import org.apache.beam.sdk.util.state.StateNamespaces.WindowAndTriggerNamespace; import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.util.state.TestInMemoryStateInternals; -import org.apache.beam.sdk.util.state.TimerCallback; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; @@ -221,14 +220,22 @@ private StateNamespace windowNamespace(W window) { * possible. */ public void advanceInputWatermark(Instant newInputWatermark) throws Exception { - // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694 - timerInternals.advanceInputWatermark(TimerCallback.NO_OP, newInputWatermark); + timerInternals.advanceInputWatermark(newInputWatermark); + while (timerInternals.removeNextEventTimer() != null) { + // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694 + } } /** Advance the processing time to the specified time. */ public void advanceProcessingTime(Instant newProcessingTime) throws Exception { - // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694 - timerInternals.advanceProcessingTime(TimerCallback.NO_OP, newProcessingTime); + timerInternals.advanceProcessingTime(newProcessingTime); + while (timerInternals.removeNextProcessingTimer() != null) { + // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694 + } + timerInternals.advanceSynchronizedProcessingTime(newProcessingTime); + while (timerInternals.removeNextSynchronizedProcessingTimer() != null) { + // TODO: Should test timer firings: see https://issues.apache.org/jira/browse/BEAM-694 + } } /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 5c6b2c19eea4..87cbbcde54f9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -47,7 +47,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -201,9 +200,7 @@ public boolean isEmpty() { // Drop any elements within expired windows reduceFnRunner.processElements( dropExpiredWindows(key, workItem.elementsIterable(), timerInternals)); - for (TimerData timer : workItem.timersIterable()) { - reduceFnRunner.onTimer(timer); - } + reduceFnRunner.onTimers(workItem.timersIterable()); reduceFnRunner.persist(); } From 42bb15d2df28b99b6788010450f41f2932095771 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 13:51:19 -0800 Subject: [PATCH 091/279] Allow stateful DoFn in DataflowRunner --- runners/google-cloud-dataflow-java/pom.xml | 1 - .../dataflow/DataflowPipelineTranslator.java | 22 +++++++------------ 2 files changed, 8 insertions(+), 15 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index adebb2a7fd8d..9ead74a17d6a 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -78,7 +78,6 @@ runnable-on-service-tests - org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesSplittableParDo diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 1cff42adf204..f43e176d0b6d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -77,6 +77,7 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Window; @@ -960,7 +961,6 @@ public void translate( private void translateMultiHelper( ParDo.BoundMulti transform, TranslationContext context) { - rejectStatefulDoFn(transform.getNewFn()); context.addStep(transform, "ParallelDo"); translateInputs(context.getInput(transform), transform.getSideInputs(), context); @@ -990,7 +990,6 @@ public void translate( private void translateSingleHelper( ParDo.Bound transform, TranslationContext context) { - rejectStatefulDoFn(transform.getNewFn()); context.addStep(transform, "ParallelDo"); translateInputs(context.getInput(transform), transform.getSideInputs(), context); @@ -1038,18 +1037,6 @@ private void translateHelper( registerTransformTranslator(Read.Bounded.class, new ReadTranslator()); } - private static void rejectStatefulDoFn(DoFn doFn) { - if (DoFnSignatures.getSignature(doFn.getClass()).isStateful()) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - DataflowRunner.class.getSimpleName())); - } - } - private static void translateInputs( PCollection input, List> sideInputs, @@ -1081,6 +1068,9 @@ private static void translateFn( TranslationContext context, long mainOutput, Map> outputMap) { + + DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); + context.addInput(PropertyNames.USER_FN, fn.getClass().getName()); context.addInput( PropertyNames.SERIALIZED_FN, @@ -1088,6 +1078,10 @@ private static void translateFn( serializeToByteArray( DoFnInfo.forFn( fn, windowingStrategy, sideInputs, inputCoder, mainOutput, outputMap)))); + + if (signature.isStateful()) { + context.addInput(PropertyNames.USES_KEYED_STATE, "true"); + } } private static BiMap> translateOutputs( From 2c8311f0eaabc130f09cb8566717ce8ea885ba97 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sat, 3 Dec 2016 13:20:24 -0800 Subject: [PATCH 092/279] [BEAM-551] Add DisplayData handling of ValueProvider --- .../sdk/transforms/display/DisplayData.java | 16 ++++ .../transforms/display/DisplayDataTest.java | 86 +++++++++++++++++++ 2 files changed, 102 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java index 1a2de970e399..f0040f75b8b1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java @@ -37,6 +37,7 @@ import java.util.Objects; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.PTransform; import org.joda.time.Duration; import org.joda.time.Instant; @@ -862,6 +863,21 @@ public static ItemSpec item(String key, @Nullable String value) { return item(key, Type.STRING, value); } + /** + * Create a display item for the specified key and {@link ValueProvider}. + */ + public static ItemSpec item(String key, ValueProvider value) { + if (value.isAccessible()) { + Object got = value.get(); + Type type = inferType(got); + if (type == null) { + throw new RuntimeException(String.format("Unknown value type: %s", got)); + } + return item(key, type, got); + } + return item(key, Type.STRING, value.toString()); + } + /** * Create a display item for the specified key and integer value. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index 770b836e3790..fccd03138759 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -53,6 +53,8 @@ import java.util.Collection; import java.util.Map; import java.util.regex.Pattern; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.DisplayData.Item; @@ -164,6 +166,90 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(data, hasDisplayItem("foo", "bar")); } + @Test + public void testStaticValueProviderDate() { + final Instant value = Instant.now(); + DisplayData data = + DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item( + "foo", StaticValueProvider.of(value))); + } + }); + + @SuppressWarnings("unchecked") + DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; + + @SuppressWarnings("unchecked") + Matcher matchesAllOf = Matchers.allOf( + hasKey("foo"), + hasType(DisplayData.Type.TIMESTAMP), + hasValue(ISO_FORMATTER.print(value))); + + assertThat(item, matchesAllOf); + } + + @Test + public void testStaticValueProviderString() { + DisplayData data = + DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item( + "foo", StaticValueProvider.of("bar"))); + } + }); + + assertThat(data.items(), hasSize(1)); + assertThat(data, hasDisplayItem("foo", "bar")); + } + + @Test + public void testStaticValueProviderInt() { + DisplayData data = + DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item( + "foo", StaticValueProvider.of(1))); + } + }); + + assertThat(data.items(), hasSize(1)); + assertThat(data, hasDisplayItem("foo", 1)); + } + + @Test + public void testInaccessibleValueProvider() { + DisplayData data = + DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item( + "foo", new ValueProvider() { + @Override + public boolean isAccessible() { + return false; + } + + @Override + public String get() { + return "bar"; + } + + @Override + public String toString() { + return "toString"; + } + })); + } + }); + + assertThat(data.items(), hasSize(1)); + assertThat(data, hasDisplayItem("foo", "toString")); + } + @Test public void testAsMap() { DisplayData data = From 5c4415e2f0ac0ad3b12b6c78c4d41160b10acc64 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sat, 3 Dec 2016 13:40:38 -0800 Subject: [PATCH 093/279] [BEAM-551] Add support for native PubSubIO --- .../beam/runners/dataflow/DataflowRunner.java | 37 +++++++++++++++---- .../beam/sdk/io/PubsubUnboundedSink.java | 4 ++ .../beam/sdk/io/PubsubUnboundedSource.java | 10 +++++ .../beam/sdk/options/ValueProvider.java | 14 +++++++ .../apache/beam/sdk/util/PropertyNames.java | 2 + .../beam/sdk/options/ValueProviderTest.java | 9 +++++ 6 files changed, 68 insertions(+), 8 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index f485cb8dbd3b..339771b39a52 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -118,6 +118,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Aggregator; @@ -2125,14 +2126,27 @@ public void translate( PubsubUnboundedSource overriddenTransform = transform.getOverriddenTransform(); context.addStep(transform, "ParallelRead"); context.addInput(PropertyNames.FORMAT, "pubsub"); - if (overriddenTransform.getTopic() != null) { - context.addInput(PropertyNames.PUBSUB_TOPIC, - overriddenTransform.getTopic().getV1Beta1Path()); + if (overriddenTransform.getTopicProvider() != null) { + if (overriddenTransform.getTopicProvider().isAccessible()) { + context.addInput( + PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path()); + } else { + context.addInput( + PropertyNames.PUBSUB_TOPIC_OVERRIDE, + ((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName()); + } } - if (overriddenTransform.getSubscription() != null) { - context.addInput( - PropertyNames.PUBSUB_SUBSCRIPTION, - overriddenTransform.getSubscription().getV1Beta1Path()); + if (overriddenTransform.getSubscriptionProvider() != null) { + if (overriddenTransform.getSubscriptionProvider().isAccessible()) { + context.addInput( + PropertyNames.PUBSUB_SUBSCRIPTION, + overriddenTransform.getSubscription().getV1Beta1Path()); + } else { + context.addInput( + PropertyNames.PUBSUB_SUBSCRIPTION_OVERRIDE, + ((NestedValueProvider) overriddenTransform.getSubscriptionProvider()) + .propertyName()); + } } if (overriddenTransform.getTimestampLabel() != null) { context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, @@ -2195,7 +2209,14 @@ public void translate( PubsubUnboundedSink overriddenTransform = transform.getOverriddenTransform(); context.addStep(transform, "ParallelWrite"); context.addInput(PropertyNames.FORMAT, "pubsub"); - context.addInput(PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path()); + if (overriddenTransform.getTopicProvider().isAccessible()) { + context.addInput( + PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path()); + } else { + context.addInput( + PropertyNames.PUBSUB_TOPIC_OVERRIDE, + ((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName()); + } if (overriddenTransform.getTimestampLabel() != null) { context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, overriddenTransform.getTimestampLabel()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java index 1e369c8393af..1a86a1c88851 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java @@ -405,6 +405,10 @@ public TopicPath getTopic() { return topic.get(); } + public ValueProvider getTopicProvider() { + return topic; + } + @Nullable public String getTimestampLabel() { return timestampLabel; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java index 4ec838924e64..cea74bce583b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java @@ -1313,11 +1313,21 @@ public TopicPath getTopic() { return topic == null ? null : topic.get(); } + @Nullable + public ValueProvider getTopicProvider() { + return topic; + } + @Nullable public SubscriptionPath getSubscription() { return subscription == null ? null : subscription.get(); } + @Nullable + public ValueProvider getSubscriptionProvider() { + return subscription; + } + @Nullable public String getTimestampLabel() { return timestampLabel; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java index 3a2e7eddf4bc..3d36a29603b9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java @@ -134,6 +134,20 @@ public boolean isAccessible() { return value.isAccessible(); } + /** + * Returns the property name associated with this provider. + */ + public String propertyName() { + if (value instanceof RuntimeValueProvider) { + return ((RuntimeValueProvider) value).propertyName(); + } else if (value instanceof NestedValueProvider) { + return ((NestedValueProvider) value).propertyName(); + } else { + throw new RuntimeException("Only a RuntimeValueProvider or a NestedValueProvider can supply" + + " a property name."); + } + } + @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java index b17bcad53b91..49a2b870c2ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java @@ -84,8 +84,10 @@ public class PropertyNames { public static final String PHASE = "phase"; public static final String PUBSUB_ID_LABEL = "pubsub_id_label"; public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription"; + public static final String PUBSUB_SUBSCRIPTION_OVERRIDE = "pubsub_subscription_runtime_override"; public static final String PUBSUB_TIMESTAMP_LABEL = "pubsub_timestamp_label"; public static final String PUBSUB_TOPIC = "pubsub_topic"; + public static final String PUBSUB_TOPIC_OVERRIDE = "pubsub_topic_runtime_override"; public static final String SCALAR_FIELD_NAME = "value"; public static final String SERIALIZED_FN = "serialized_fn"; public static final String SHARD_NAME_TEMPLATE = "shard_template"; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java index 31532b944fec..7ec40be21310 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java @@ -246,6 +246,15 @@ public String apply(String from) { return from + "bar"; } }); + ValueProvider doubleNvp = NestedValueProvider.of( + nvp, new SerializableFunction() { + @Override + public String apply(String from) { + return from; + } + }); + assertEquals("bar", ((NestedValueProvider) nvp).propertyName()); + assertEquals("bar", ((NestedValueProvider) doubleNvp).propertyName()); assertFalse(nvp.isAccessible()); expectedException.expect(RuntimeException.class); expectedException.expectMessage("Not called from a runtime context"); From e9f254ef2769a082c7fbb500c1c28c6224ac5a7f Mon Sep 17 00:00:00 2001 From: Jakob Homan Date: Tue, 6 Dec 2016 16:59:50 -0800 Subject: [PATCH 094/279] [BEAM-1099] Minor typos in KafkaIO --- .../java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 4212d59f57ed..c87d12b8dac8 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -106,7 +106,7 @@ * metadata like topic-partition and offset, along with key and value associated with a Kafka * record. * - *

      Although most applications consumer single topic, the source can be configured to consume + *

      Although most applications consume a single topic, the source can be configured to consume * multiple topics or even a specific set of {@link TopicPartition}s. * *

      To configure a Kafka source, you must specify at the minimum Kafka bootstrapServers @@ -250,8 +250,8 @@ public Read withBootstrapServers(String bootstrapServers) { } /** - * Returns a new {@link Read} that reads from the topics. All the partitions are from each - * of the topics is read. + * Returns a new {@link Read} that reads from the topics. All the partitions from each + * of the topics are read. * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions)} for description * of how the partitions are distributed among the splits. */ @@ -1134,7 +1134,7 @@ public void close() throws IOException { boolean isShutdown = false; - // Wait for threads to shutdown. Trying this a loop to handle a tiny race where poll thread + // Wait for threads to shutdown. Trying this as a loop to handle a tiny race where poll thread // might block to enqueue right after availableRecordsQueue.poll() below. while (!isShutdown) { @@ -1178,7 +1178,7 @@ public Write withBootstrapServers(String bootstrapServers) { } /** - * Returns a new {@link Write} transform that write to given topic. + * Returns a new {@link Write} transform that writes to given topic. */ public Write withTopic(String topic) { return new Write(topic, keyCoder, valueCoder, producerConfig); From ded58832ceaef487f4590d9396f09744288c955d Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 23 Nov 2016 16:14:27 -0800 Subject: [PATCH 095/279] [Code Health] Remove redundant projectId from DataflowPipelineJob. --- .../runners/dataflow/DataflowPipelineJob.java | 22 +++------ .../beam/runners/dataflow/DataflowRunner.java | 4 +- .../dataflow/util/DataflowTemplateJob.java | 2 +- .../dataflow/DataflowPipelineJobTest.java | 48 +++++++++---------- .../testing/TestDataflowRunnerTest.java | 36 +++++--------- 5 files changed, 45 insertions(+), 67 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index a2b632fcc7a2..58e85e09c9ec 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -61,11 +61,6 @@ public class DataflowPipelineJob implements PipelineResult { */ private String jobId; - /** - * Google cloud project to associate this pipeline with. - */ - private String projectId; - /** * Client for the Dataflow service. This can be used to query the service * for information about the job. @@ -119,17 +114,14 @@ public class DataflowPipelineJob implements PipelineResult { /** * Constructs the job. * - * @param projectId the project id * @param jobId the job id * @param dataflowOptions used to configure the client for the Dataflow Service * @param aggregatorTransforms a mapping from aggregators to PTransforms */ public DataflowPipelineJob( - String projectId, String jobId, DataflowPipelineOptions dataflowOptions, DataflowAggregatorTransforms aggregatorTransforms) { - this.projectId = projectId; this.jobId = jobId; this.dataflowOptions = dataflowOptions; this.aggregatorTransforms = aggregatorTransforms; @@ -146,7 +138,7 @@ public String getJobId() { * Get the project this job exists in. */ public String getProjectId() { - return projectId; + return dataflowOptions.getProject(); } /** @@ -249,7 +241,7 @@ State waitUntilFinish( MonitoringUtil.JobMessagesHandler messageHandler, Sleeper sleeper, NanoClock nanoClock) throws IOException, InterruptedException { - MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowOptions.getDataflowClient()); + MonitoringUtil monitor = new MonitoringUtil(getProjectId(), dataflowOptions.getDataflowClient()); long lastTimestamp = 0; BackOff backoff; @@ -338,12 +330,12 @@ State waitUntilFinish( @Override public State cancel() throws IOException { Job content = new Job(); - content.setProjectId(projectId); + content.setProjectId(getProjectId()); content.setId(jobId); content.setRequestedState("JOB_STATE_CANCELLED"); try { dataflowOptions.getDataflowClient().projects().jobs() - .update(projectId, jobId, content) + .update(getProjectId(), jobId, content) .execute(); return State.CANCELLED; } catch (IOException e) { @@ -412,13 +404,13 @@ private Job getJobWithRetries(BackOff backoff, Sleeper sleeper) throws IOExcepti Job job = dataflowOptions.getDataflowClient() .projects() .jobs() - .get(projectId, jobId) + .get(getProjectId(), jobId) .execute(); State currentState = MonitoringUtil.toState(job.getCurrentState()); if (currentState.isTerminal()) { terminalState = currentState; replacedByJob = new DataflowPipelineJob( - getProjectId(), job.getReplacedByJobId(), dataflowOptions, aggregatorTransforms); + job.getReplacedByJobId(), dataflowOptions, aggregatorTransforms); } return job; } catch (IOException exn) { @@ -485,7 +477,7 @@ private Map fromMetricUpdates(Aggregator } else { boolean terminal = getState().isTerminal(); JobMetrics jobMetrics = dataflowOptions.getDataflowClient() - .projects().jobs().getMetrics(projectId, jobId).execute(); + .projects().jobs().getMetrics(getProjectId(), jobId).execute(); metricUpdates = jobMetrics.getMetrics(); if (terminal && jobMetrics.getMetrics() != null) { terminalMetricUpdates = metricUpdates; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 339771b39a52..e781b4e897c0 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -629,8 +629,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Use a raw client for post-launch monitoring, as status calls may fail // regularly and need not be retried automatically. - DataflowPipelineJob dataflowPipelineJob = new DataflowPipelineJob( - options.getProject(), jobResult.getId(), options, aggregatorTransforms); + DataflowPipelineJob dataflowPipelineJob = + new DataflowPipelineJob(jobResult.getId(), options, aggregatorTransforms); // If the service returned client request id, the SDK needs to compare it // with the original id generated in the request, if they are not the same diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java index 2937184090a8..1a44963e4e78 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java @@ -30,7 +30,7 @@ public class DataflowTemplateJob extends DataflowPipelineJob { "The result of template creation should not be used."; public DataflowTemplateJob() { - super(null, null, null, null); + super(null, null, null); } @Override diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index 0527b7c2edb5..323f762d6e95 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -115,6 +115,7 @@ public void setup() { options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class); options.setDataflowClient(mockWorkflowClient); + options.setProject(PROJECT_ID); } /** @@ -160,8 +161,8 @@ public void testWaitToFinishMessagesFail() throws Exception { DataflowAggregatorTransforms dataflowAggregatorTransforms = mock(DataflowAggregatorTransforms.class); - DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + DataflowPipelineJob job = + new DataflowPipelineJob(JOB_ID, options, dataflowAggregatorTransforms); State state = job.waitUntilFinish( Duration.standardMinutes(5), jobHandler, fastClock, fastClock); @@ -182,8 +183,8 @@ public State mockWaitToFinishInState(State state) throws Exception { DataflowAggregatorTransforms dataflowAggregatorTransforms = mock(DataflowAggregatorTransforms.class); - DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + DataflowPipelineJob job = + new DataflowPipelineJob(JOB_ID, options, dataflowAggregatorTransforms); return job.waitUntilFinish(Duration.standardMinutes(1), null, fastClock, fastClock); } @@ -249,8 +250,8 @@ public void testWaitToFinishFail() throws Exception { DataflowAggregatorTransforms dataflowAggregatorTransforms = mock(DataflowAggregatorTransforms.class); - DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + DataflowPipelineJob job = + new DataflowPipelineJob(JOB_ID, options, dataflowAggregatorTransforms); long startTime = fastClock.nanoTime(); State state = job.waitUntilFinish(Duration.standardMinutes(5), null, fastClock, fastClock); @@ -269,8 +270,8 @@ public void testWaitToFinishTimeFail() throws Exception { DataflowAggregatorTransforms dataflowAggregatorTransforms = mock(DataflowAggregatorTransforms.class); - DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + DataflowPipelineJob job = + new DataflowPipelineJob(JOB_ID, options, dataflowAggregatorTransforms); long startTime = fastClock.nanoTime(); State state = job.waitUntilFinish(Duration.millis(4), null, fastClock, fastClock); assertEquals(null, state); @@ -294,7 +295,7 @@ public void testCumulativeTimeOverflow() throws Exception { FastNanoClockAndFuzzySleeper clock = new FastNanoClockAndFuzzySleeper(); DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + JOB_ID, options, dataflowAggregatorTransforms); long startTime = clock.nanoTime(); State state = job.waitUntilFinish(Duration.millis(4), null, clock, clock); assertEquals(null, state); @@ -317,7 +318,7 @@ public void testGetStateReturnsServiceState() throws Exception { mock(DataflowAggregatorTransforms.class); DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + JOB_ID, options, dataflowAggregatorTransforms); assertEquals( State.RUNNING, @@ -333,8 +334,8 @@ public void testGetStateWithExceptionReturnsUnknown() throws Exception { DataflowAggregatorTransforms dataflowAggregatorTransforms = mock(DataflowAggregatorTransforms.class); - DataflowPipelineJob job = new DataflowPipelineJob( - PROJECT_ID, JOB_ID, options, dataflowAggregatorTransforms); + DataflowPipelineJob job = + new DataflowPipelineJob(JOB_ID, options, dataflowAggregatorTransforms); long startTime = fastClock.nanoTime(); assertEquals( @@ -373,7 +374,7 @@ public void testGetAggregatorValuesWithNoMetricUpdatesReturnsEmptyValue() modelJob.setCurrentState(State.RUNNING.toString()); DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); AggregatorValues values = job.getAggregatorValues(aggregator); @@ -408,7 +409,7 @@ public void testGetAggregatorValuesWithNullMetricUpdatesReturnsEmptyValue() modelJob.setCurrentState(State.RUNNING.toString()); DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); AggregatorValues values = job.getAggregatorValues(aggregator); @@ -453,8 +454,7 @@ public void testGetAggregatorValuesWithSingleMetricUpdateReturnsSingletonCollect when(getState.execute()).thenReturn(modelJob); modelJob.setCurrentState(State.RUNNING.toString()); - DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); AggregatorValues values = job.getAggregatorValues(aggregator); @@ -521,8 +521,7 @@ public void testGetAggregatorValuesWithMultipleMetricUpdatesReturnsCollection() when(getState.execute()).thenReturn(modelJob); modelJob.setCurrentState(State.RUNNING.toString()); - DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); AggregatorValues values = job.getAggregatorValues(aggregator); @@ -571,7 +570,7 @@ public void testGetAggregatorValuesWithUnrelatedMetricUpdateIgnoresUpdate() modelJob.setCurrentState(State.RUNNING.toString()); DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); AggregatorValues values = job.getAggregatorValues(aggregator); @@ -589,7 +588,7 @@ public void testGetAggregatorValuesWithUnusedAggregatorThrowsException() ImmutableMap., String>of()); DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("not used in this pipeline"); @@ -624,8 +623,7 @@ public void testGetAggregatorValuesWhenClientThrowsExceptionThrowsAggregatorRetr when(getState.execute()).thenReturn(modelJob); modelJob.setCurrentState(State.RUNNING.toString()); - DataflowPipelineJob job = - new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, aggregatorTransforms); + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, aggregatorTransforms); thrown.expect(AggregatorRetrievalException.class); thrown.expectCause(is(cause)); @@ -690,7 +688,7 @@ public void testCancelUnterminatedJobThatSucceeds() throws IOException { when(mockJobs.update(anyString(), anyString(), any(Job.class))).thenReturn(update); when(update.execute()).thenReturn(new Job()); - DataflowPipelineJob job = new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, null); + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, null); assertEquals(State.CANCELLED, job.cancel()); Job content = new Job(); @@ -714,7 +712,7 @@ public void testCancelUnterminatedJobThatFails() throws IOException { when(mockJobs.update(anyString(), anyString(), any(Job.class))).thenReturn(update); when(update.execute()).thenThrow(new IOException()); - DataflowPipelineJob job = new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, null); + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, null); thrown.expect(IOException.class); thrown.expectMessage("Failed to cancel the job, " @@ -742,7 +740,7 @@ public void testCancelTerminatedJob() throws IOException { when(mockJobs.update(anyString(), anyString(), any(Job.class))).thenReturn(update); when(update.execute()).thenThrow(new IOException()); - DataflowPipelineJob job = new DataflowPipelineJob(PROJECT_ID, JOB_ID, options, null); + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, null); assertEquals(State.FAILED, job.cancel()); Job content = new Job(); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java index e6b513a9a65a..366c6a13835c 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java @@ -344,8 +344,7 @@ private JobMetrics buildJobMetrics(List metricList) { @Test public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); @@ -359,8 +358,7 @@ public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception { @Test public void testCheckingForSuccessWhenPAssertFails() throws Exception { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); @@ -374,8 +372,7 @@ public void testCheckingForSuccessWhenPAssertFails() throws Exception { @Test public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); @@ -389,8 +386,7 @@ public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception { @Test public void testCheckMaxWatermarkWithNoWatermarkMetric() throws IOException { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -403,8 +399,7 @@ public void testCheckMaxWatermarkWithNoWatermarkMetric() throws IOException { @Test public void testCheckMaxWatermarkWithSingleWatermarkAtMax() throws IOException { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -417,8 +412,7 @@ public void testCheckMaxWatermarkWithSingleWatermarkAtMax() throws IOException { @Test public void testCheckMaxWatermarkWithSingleWatermarkNotAtMax() throws IOException { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -431,8 +425,7 @@ public void testCheckMaxWatermarkWithSingleWatermarkNotAtMax() throws IOExceptio @Test public void testCheckMaxWatermarkWithMultipleWatermarksAtMax() throws IOException { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -446,8 +439,7 @@ public void testCheckMaxWatermarkWithMultipleWatermarksAtMax() throws IOExceptio @Test public void testCheckMaxWatermarkWithMultipleMaxAndNotMaxWatermarks() throws IOException { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -461,8 +453,7 @@ public void testCheckMaxWatermarkWithMultipleMaxAndNotMaxWatermarks() throws IOE @Test public void testCheckMaxWatermarkIgnoresUnrelatedMatrics() throws IOException { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -476,8 +467,7 @@ public void testCheckMaxWatermarkIgnoresUnrelatedMatrics() throws IOException { @Test public void testStreamingPipelineFailsIfServiceFails() throws Exception { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); PCollection pc = p.apply(Create.of(1, 2, 3)); PAssert.that(pc).containsInAnyOrder(1, 2, 3); @@ -532,8 +522,7 @@ public State answer(InvocationOnMock invocation) { @Test public void testGetJobMetricsThatSucceeds() throws Exception { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); @@ -549,8 +538,7 @@ public void testGetJobMetricsThatSucceeds() throws Exception { @Test public void testGetJobMetricsThatFailsForException() throws Exception { - DataflowPipelineJob job = - spy(new DataflowPipelineJob("test-project", "test-job", options, null)); + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); Pipeline p = TestPipeline.create(options); p.apply(Create.of(1, 2, 3)); From e8c9686a2e898d38afd692328eb171c542084747 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 23 Nov 2016 15:59:56 -0800 Subject: [PATCH 096/279] [BEAM-1047] Add DataflowClient wrapper on top of JSON library. --- .../beam/runners/dataflow/DataflowClient.java | 130 ++++++++++++++++++ 1 file changed, 130 insertions(+) create mode 100644 runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java new file mode 100644 index 000000000000..f2081dba304a --- /dev/null +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java @@ -0,0 +1,130 @@ +/* + * 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.dataflow; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.LeaseWorkItemRequest; +import com.google.api.services.dataflow.model.LeaseWorkItemResponse; +import com.google.api.services.dataflow.model.ListJobMessagesResponse; +import com.google.api.services.dataflow.model.ListJobsResponse; +import com.google.api.services.dataflow.model.ReportWorkItemStatusRequest; +import com.google.api.services.dataflow.model.ReportWorkItemStatusResponse; +import java.io.IOException; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; + +/** + * Client library for {@link Dataflow}. + */ +public class DataflowClient { + + public static DataflowClient create(DataflowPipelineOptions options) { + return new DataflowClient(options.getDataflowClient(), options); + } + + private final Dataflow dataflow; + private final DataflowPipelineOptions options; + + private DataflowClient(Dataflow dataflow, DataflowPipelineOptions options) { + this.dataflow = checkNotNull(dataflow, "dataflow"); + this.options = checkNotNull(options, "options"); + } + + /** + * Creates the Dataflow {@link Job}. + */ + public Job createJob(@Nonnull Job job) throws IOException { + Jobs.Create jobsCreate = dataflow.projects().jobs().create(options.getProject(), job); + return jobsCreate.execute(); + } + + /** + * Lists Dataflow {@link Job Jobs} in the project associated with + * the {@link DataflowPipelineOptions}. + */ + public ListJobsResponse listJobs(@Nullable String pageToken) throws IOException { + Jobs.List jobsList = dataflow.projects().jobs() + .list(options.getProject()) + .setPageToken(pageToken); + return jobsList.execute(); + } + + /** + * Updates the Dataflow {@link Job} with the given {@code jobId}. + */ + public Job updateJob(@Nonnull String jobId, @Nonnull Job content) throws IOException { + Jobs.Update jobsUpdate = dataflow.projects().jobs() + .update(options.getProject(), jobId, content); + return jobsUpdate.execute(); + } + + /** + * Gets the Dataflow {@link Job} with the given {@code jobId}. + */ + public Job getJob(@Nonnull String jobId) throws IOException { + Jobs.Get jobsGet = dataflow.projects().jobs() + .get(options.getProject(), jobId); + return jobsGet.execute(); + } + + /** + * Gets the {@link JobMetrics} with the given {@code jobId}. + */ + public JobMetrics getJobMetrics(@Nonnull String jobId) throws IOException { + Jobs.GetMetrics jobsGetMetrics = dataflow.projects().jobs() + .getMetrics(options.getProject(), jobId); + return jobsGetMetrics.execute(); + } + + /** + * Lists job messages with the given {@code jobId}. + */ + public ListJobMessagesResponse listJobMessages( + @Nonnull String jobId, @Nullable String pageToken) throws IOException { + Jobs.Messages.List jobMessagesList = dataflow.projects().jobs().messages() + .list(options.getProject(), jobId) + .setPageToken(pageToken); + return jobMessagesList.execute(); + } + + /** + * Leases the work item for {@code jobId}. + */ + public LeaseWorkItemResponse leaseWorkItem( + @Nonnull String jobId, @Nonnull LeaseWorkItemRequest request) throws IOException { + Jobs.WorkItems.Lease jobWorkItemsLease = dataflow.projects().jobs().workItems() + .lease(options.getProject(), jobId, request); + return jobWorkItemsLease.execute(); + } + + /** + * Reports the status of the work item for {@code jobId}. + */ + public ReportWorkItemStatusResponse reportWorkItemStatus( + @Nonnull String jobId, @Nonnull ReportWorkItemStatusRequest request) throws IOException { + Jobs.WorkItems.ReportStatus jobWorkItemsReportStatus = dataflow.projects().jobs().workItems() + .reportStatus(options.getProject(), jobId, request); + return jobWorkItemsReportStatus.execute(); + } +} From ce03f30c1ee0b84ad2e7f10a6272ffb25548244a Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 28 Nov 2016 11:47:42 -0800 Subject: [PATCH 097/279] [BEAM-1047] Update dataflow runner code to use DataflowClient wrapper. --- .../beam/runners/dataflow/DataflowClient.java | 36 ++++++++++++------- .../runners/dataflow/DataflowPipelineJob.java | 23 ++++++------ .../beam/runners/dataflow/DataflowRunner.java | 16 +++------ .../dataflow/testing/TestDataflowRunner.java | 6 ++-- .../runners/dataflow/util/MonitoringUtil.java | 22 +++--------- .../dataflow/DataflowPipelineJobTest.java | 1 + .../transforms/DataflowGroupByKeyTest.java | 14 +++++++- .../dataflow/transforms/DataflowViewTest.java | 16 +++++++-- .../dataflow/util/MonitoringUtilTest.java | 21 +++-------- 9 files changed, 80 insertions(+), 75 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java index f2081dba304a..3536d7272c3a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java @@ -35,27 +35,28 @@ import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; /** - * Client library for {@link Dataflow}. + * Wrapper around the generated {@link Dataflow} client to provide common functionality. */ public class DataflowClient { public static DataflowClient create(DataflowPipelineOptions options) { - return new DataflowClient(options.getDataflowClient(), options); + return new DataflowClient(options.getDataflowClient(), options.getProject()); } private final Dataflow dataflow; - private final DataflowPipelineOptions options; + private final String projectId; - private DataflowClient(Dataflow dataflow, DataflowPipelineOptions options) { + private DataflowClient(Dataflow dataflow, String projectId) { this.dataflow = checkNotNull(dataflow, "dataflow"); - this.options = checkNotNull(options, "options"); + this.projectId = checkNotNull(projectId, "options"); } /** * Creates the Dataflow {@link Job}. */ public Job createJob(@Nonnull Job job) throws IOException { - Jobs.Create jobsCreate = dataflow.projects().jobs().create(options.getProject(), job); + checkNotNull(job, "job"); + Jobs.Create jobsCreate = dataflow.projects().jobs().create(projectId, job); return jobsCreate.execute(); } @@ -65,7 +66,7 @@ public Job createJob(@Nonnull Job job) throws IOException { */ public ListJobsResponse listJobs(@Nullable String pageToken) throws IOException { Jobs.List jobsList = dataflow.projects().jobs() - .list(options.getProject()) + .list(projectId) .setPageToken(pageToken); return jobsList.execute(); } @@ -74,8 +75,10 @@ public ListJobsResponse listJobs(@Nullable String pageToken) throws IOException * Updates the Dataflow {@link Job} with the given {@code jobId}. */ public Job updateJob(@Nonnull String jobId, @Nonnull Job content) throws IOException { + checkNotNull(jobId, "jobId"); + checkNotNull(content, "content"); Jobs.Update jobsUpdate = dataflow.projects().jobs() - .update(options.getProject(), jobId, content); + .update(projectId, jobId, content); return jobsUpdate.execute(); } @@ -83,8 +86,9 @@ public Job updateJob(@Nonnull String jobId, @Nonnull Job content) throws IOExcep * Gets the Dataflow {@link Job} with the given {@code jobId}. */ public Job getJob(@Nonnull String jobId) throws IOException { + checkNotNull(jobId, "jobId"); Jobs.Get jobsGet = dataflow.projects().jobs() - .get(options.getProject(), jobId); + .get(projectId, jobId); return jobsGet.execute(); } @@ -92,8 +96,9 @@ public Job getJob(@Nonnull String jobId) throws IOException { * Gets the {@link JobMetrics} with the given {@code jobId}. */ public JobMetrics getJobMetrics(@Nonnull String jobId) throws IOException { + checkNotNull(jobId, "jobId"); Jobs.GetMetrics jobsGetMetrics = dataflow.projects().jobs() - .getMetrics(options.getProject(), jobId); + .getMetrics(projectId, jobId); return jobsGetMetrics.execute(); } @@ -102,8 +107,9 @@ public JobMetrics getJobMetrics(@Nonnull String jobId) throws IOException { */ public ListJobMessagesResponse listJobMessages( @Nonnull String jobId, @Nullable String pageToken) throws IOException { + checkNotNull(jobId, "jobId"); Jobs.Messages.List jobMessagesList = dataflow.projects().jobs().messages() - .list(options.getProject(), jobId) + .list(projectId, jobId) .setPageToken(pageToken); return jobMessagesList.execute(); } @@ -113,8 +119,10 @@ public ListJobMessagesResponse listJobMessages( */ public LeaseWorkItemResponse leaseWorkItem( @Nonnull String jobId, @Nonnull LeaseWorkItemRequest request) throws IOException { + checkNotNull(jobId, "jobId"); + checkNotNull(request, "request"); Jobs.WorkItems.Lease jobWorkItemsLease = dataflow.projects().jobs().workItems() - .lease(options.getProject(), jobId, request); + .lease(projectId, jobId, request); return jobWorkItemsLease.execute(); } @@ -123,8 +131,10 @@ public LeaseWorkItemResponse leaseWorkItem( */ public ReportWorkItemStatusResponse reportWorkItemStatus( @Nonnull String jobId, @Nonnull ReportWorkItemStatusRequest request) throws IOException { + checkNotNull(jobId, "jobId"); + checkNotNull(request, "request"); Jobs.WorkItems.ReportStatus jobWorkItemsReportStatus = dataflow.projects().jobs().workItems() - .reportStatus(options.getProject(), jobId, request); + .reportStatus(projectId, jobId, request); return jobWorkItemsReportStatus.execute(); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 58e85e09c9ec..00c88f994958 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -61,11 +61,16 @@ public class DataflowPipelineJob implements PipelineResult { */ private String jobId; + /** + * The {@link DataflowPipelineOptions} for the job. + */ + private final DataflowPipelineOptions dataflowOptions; + /** * Client for the Dataflow service. This can be used to query the service * for information about the job. */ - private DataflowPipelineOptions dataflowOptions; + private final DataflowClient dataflowClient; /** * The state the job terminated in or {@code null} if the job has not terminated. @@ -124,6 +129,7 @@ public DataflowPipelineJob( DataflowAggregatorTransforms aggregatorTransforms) { this.jobId = jobId; this.dataflowOptions = dataflowOptions; + this.dataflowClient = (dataflowOptions == null ? null : DataflowClient.create(dataflowOptions)); this.aggregatorTransforms = aggregatorTransforms; } @@ -241,7 +247,7 @@ State waitUntilFinish( MonitoringUtil.JobMessagesHandler messageHandler, Sleeper sleeper, NanoClock nanoClock) throws IOException, InterruptedException { - MonitoringUtil monitor = new MonitoringUtil(getProjectId(), dataflowOptions.getDataflowClient()); + MonitoringUtil monitor = new MonitoringUtil(dataflowClient); long lastTimestamp = 0; BackOff backoff; @@ -334,9 +340,7 @@ public State cancel() throws IOException { content.setId(jobId); content.setRequestedState("JOB_STATE_CANCELLED"); try { - dataflowOptions.getDataflowClient().projects().jobs() - .update(getProjectId(), jobId, content) - .execute(); + dataflowClient.updateJob(jobId, content); return State.CANCELLED; } catch (IOException e) { State state = getState(); @@ -401,11 +405,7 @@ private Job getJobWithRetries(BackOff backoff, Sleeper sleeper) throws IOExcepti // Retry loop ends in return or throw while (true) { try { - Job job = dataflowOptions.getDataflowClient() - .projects() - .jobs() - .get(getProjectId(), jobId) - .execute(); + Job job = dataflowClient.getJob(jobId); State currentState = MonitoringUtil.toState(job.getCurrentState()); if (currentState.isTerminal()) { terminalState = currentState; @@ -476,8 +476,7 @@ private Map fromMetricUpdates(Aggregator metricUpdates = terminalMetricUpdates; } else { boolean terminal = getState().isTerminal(); - JobMetrics jobMetrics = dataflowOptions.getDataflowClient() - .projects().jobs().getMetrics(getProjectId(), jobId).execute(); + JobMetrics jobMetrics = dataflowClient.getJobMetrics(jobId); metricUpdates = jobMetrics.getMetrics(); if (terminal && jobMetrics.getMetrics() != null) { terminalMetricUpdates = metricUpdates; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index e781b4e897c0..40d894800710 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -32,7 +32,6 @@ import com.google.api.services.clouddebugger.v2.model.Debuggee; import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest; import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse; -import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; @@ -194,7 +193,7 @@ public class DataflowRunner extends PipelineRunner { private final DataflowPipelineOptions options; /** Client for the Dataflow service. This is used to actually submit jobs. */ - private final Dataflow dataflowClient; + private final DataflowClient dataflowClient; /** Translator for this DataflowRunner, based on options. */ private final DataflowPipelineTranslator translator; @@ -321,7 +320,7 @@ public static DataflowRunner fromOptions(PipelineOptions options) { @VisibleForTesting protected DataflowRunner(DataflowPipelineOptions options) { this.options = options; - this.dataflowClient = options.getDataflowClient(); + this.dataflowClient = DataflowClient.create(options); this.translator = DataflowPipelineTranslator.fromOptions(options); this.pcollectionsRequiringIndexedFormat = new HashSet<>(); this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>(); @@ -597,11 +596,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { } Job jobResult; try { - jobResult = dataflowClient - .projects() - .jobs() - .create(options.getProject(), newJob) - .execute(); + jobResult = dataflowClient.createJob(newJob); } catch (GoogleJsonResponseException e) { String errorMessages = "Unexpected errors"; if (e.getDetails() != null) { @@ -2830,10 +2825,7 @@ private String getJobIdFromName(String jobName) { ListJobsResponse listResult; String token = null; do { - listResult = dataflowClient.projects().jobs() - .list(options.getProject()) - .setPageToken(token) - .execute(); + listResult = dataflowClient.listJobs(token); token = listResult.getNextPageToken(); for (Job job : listResult.getJobs()) { if (job.getName().equals(jobName) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java index 70c3f588f4de..4b0fcf218a75 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java @@ -34,6 +34,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowClient; import org.apache.beam.runners.dataflow.DataflowPipelineJob; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.util.MonitoringUtil; @@ -65,11 +66,13 @@ public class TestDataflowRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class); private final TestDataflowPipelineOptions options; + private final DataflowClient dataflowClient; private final DataflowRunner runner; private int expectedNumberOfAssertions = 0; TestDataflowRunner(TestDataflowPipelineOptions options) { this.options = options; + this.dataflowClient = DataflowClient.create(options); this.runner = DataflowRunner.fromOptions(options); } @@ -279,8 +282,7 @@ boolean atMaxWatermark(DataflowPipelineJob job, JobMetrics metrics) { JobMetrics getJobMetrics(DataflowPipelineJob job) { JobMetrics metrics = null; try { - metrics = options.getDataflowClient().projects().jobs() - .getMetrics(job.getProjectId(), job.getJobId()).execute(); + metrics = dataflowClient.getJobMetrics(job.getJobId()); } catch (IOException e) { LOG.warn("Failed to get job metrics: ", e); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java index efb6d2b224f0..d0a24bf0ec30 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java @@ -20,7 +20,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages; import com.google.api.services.dataflow.model.JobMessage; import com.google.api.services.dataflow.model.ListJobMessagesResponse; import com.google.common.base.MoreObjects; @@ -35,6 +34,7 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowClient; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.PipelineResult.State; import org.joda.time.Instant; @@ -67,8 +67,7 @@ public final class MonitoringUtil { private static final String JOB_MESSAGE_DETAILED = "JOB_MESSAGE_DETAILED"; private static final String JOB_MESSAGE_DEBUG = "JOB_MESSAGE_DEBUG"; - private String projectId; - private Messages messagesClient; + private final DataflowClient dataflowClient; /** * An interface that can be used for defining callbacks to receive a list @@ -115,14 +114,8 @@ public void process(List messages) { } /** Construct a helper for monitoring. */ - public MonitoringUtil(String projectId, Dataflow dataflow) { - this(projectId, dataflow.projects().jobs().messages()); - } - - // @VisibleForTesting - MonitoringUtil(String projectId, Messages messagesClient) { - this.projectId = projectId; - this.messagesClient = messagesClient; + public MonitoringUtil(DataflowClient dataflowClient) { + this.dataflowClient = dataflowClient; } /** @@ -157,12 +150,7 @@ public ArrayList getJobMessages( ArrayList allMessages = new ArrayList<>(); String pageToken = null; while (true) { - Messages.List listRequest = messagesClient.list(projectId, jobId); - if (pageToken != null) { - listRequest.setPageToken(pageToken); - } - ListJobMessagesResponse response = listRequest.execute(); - + ListJobMessagesResponse response = dataflowClient.listJobMessages(jobId, pageToken); if (response == null || response.getJobMessages() == null) { return allMessages; } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index 323f762d6e95..1890da1cb58d 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -157,6 +157,7 @@ public void testWaitToFinishMessagesFail() throws Exception { Messages.List listRequest = mock(Dataflow.Projects.Jobs.Messages.List.class); when(mockJobs.messages()).thenReturn(mockMessages); when(mockMessages.list(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(listRequest); + when(listRequest.setPageToken(eq((String) null))).thenReturn(listRequest); when(listRequest.execute()).thenThrow(SocketTimeoutException.class); DataflowAggregatorTransforms dataflowAggregatorTransforms = mock(DataflowAggregatorTransforms.class); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java index bb84d9855054..67408ae43574 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.dataflow.transforms; +import com.google.api.services.dataflow.Dataflow; import java.util.Arrays; import java.util.List; import org.apache.beam.runners.dataflow.DataflowRunner; @@ -38,11 +39,14 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; /** Tests for {@link GroupByKey} for the {@link DataflowRunner}. */ @RunWith(JUnit4.class) @@ -50,6 +54,14 @@ public class DataflowGroupByKeyTest { @Rule public ExpectedException thrown = ExpectedException.none(); + @Mock + private Dataflow dataflow; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + /** * Create a test pipeline that uses the {@link DataflowRunner} so that {@link GroupByKey} * is not expanded. This is used for verifying that even without expansion the proper errors show @@ -61,7 +73,7 @@ private Pipeline createTestServiceRunner() { options.setProject("someproject"); options.setGcpTempLocation("gs://staging"); options.setPathValidatorClass(NoopPathValidator.class); - options.setDataflowClient(null); + options.setDataflowClient(dataflow); return Pipeline.create(options); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java index ed3f2cd1c89f..b9220aff20eb 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.dataflow.transforms; +import com.google.api.services.dataflow.Dataflow; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; @@ -36,12 +37,15 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.hamcrest.Matchers; import org.joda.time.Duration; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; /** Tests for {@link View} for a {@link DataflowRunner}. */ @RunWith(JUnit4.class) @@ -49,13 +53,21 @@ public class DataflowViewTest { @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Mock + private Dataflow dataflow; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + private Pipeline createTestBatchRunner() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setRunner(DataflowRunner.class); options.setProject("someproject"); options.setGcpTempLocation("gs://staging"); options.setPathValidatorClass(NoopPathValidator.class); - options.setDataflowClient(null); + options.setDataflowClient(dataflow); return Pipeline.create(options); } @@ -66,7 +78,7 @@ private Pipeline createTestStreamingRunner() { options.setProject("someproject"); options.setGcpTempLocation("gs://staging"); options.setPathValidatorClass(NoopPathValidator.class); - options.setDataflowClient(null); + options.setDataflowClient(dataflow); return Pipeline.create(options); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java index 6c5a2beb6fa6..23ed26f2d7cb 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java @@ -19,16 +19,15 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.JobMessage; import com.google.api.services.dataflow.model.ListJobMessagesResponse; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import org.apache.beam.runners.dataflow.DataflowClient; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil.LoggingHandler; import org.apache.beam.sdk.PipelineResult.State; @@ -57,15 +56,7 @@ public class MonitoringUtilTest { @Test public void testGetJobMessages() throws IOException { - Dataflow.Projects.Jobs.Messages mockMessages = mock(Dataflow.Projects.Jobs.Messages.class); - - // Two requests are needed to get all the messages. - Dataflow.Projects.Jobs.Messages.List firstRequest = - mock(Dataflow.Projects.Jobs.Messages.List.class); - Dataflow.Projects.Jobs.Messages.List secondRequest = - mock(Dataflow.Projects.Jobs.Messages.List.class); - - when(mockMessages.list(PROJECT_ID, JOB_ID)).thenReturn(firstRequest).thenReturn(secondRequest); + DataflowClient dataflowClient = mock(DataflowClient.class); ListJobMessagesResponse firstResponse = new ListJobMessagesResponse(); firstResponse.setJobMessages(new ArrayList()); @@ -87,15 +78,13 @@ public void testGetJobMessages() throws IOException { secondResponse.getJobMessages().add(message); } - when(firstRequest.execute()).thenReturn(firstResponse); - when(secondRequest.execute()).thenReturn(secondResponse); + when(dataflowClient.listJobMessages(JOB_ID, null)).thenReturn(firstResponse); + when(dataflowClient.listJobMessages(JOB_ID, pageToken)).thenReturn(secondResponse); - MonitoringUtil util = new MonitoringUtil(PROJECT_ID, mockMessages); + MonitoringUtil util = new MonitoringUtil(dataflowClient); List messages = util.getJobMessages(JOB_ID, -1); - verify(secondRequest).setPageToken(pageToken); - assertEquals(150, messages.size()); } From 0a2ed832ce5af7556db605e99b985ed4ffc1b152 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sun, 30 Oct 2016 11:58:44 -0700 Subject: [PATCH 098/279] BigQueryIO.Read: support runtime options --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 208 +++++++++++++----- .../beam/sdk/io/gcp/ApiSurfaceTest.java | 2 + .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 33 +-- 3 files changed, 176 insertions(+), 67 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index c00c19d126f1..8bfbd5355eff 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -89,6 +89,9 @@ import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Create; @@ -318,6 +321,38 @@ public static String toTableSpec(TableReference ref) { return sb.toString(); } + private static class JsonTableRefToTableRef + implements SerializableFunction { + @Override + public TableReference apply(String from) { + return fromJsonString(from, TableReference.class); + } + } + + private static class TableRefToJson + implements SerializableFunction { + @Override + public String apply(TableReference from) { + return toJsonString(from); + } + } + + private static class TableRefToProjectId + implements SerializableFunction { + @Override + public String apply(TableReference from) { + return from.getProjectId(); + } + } + + private static class TableSpecToTableRef + implements SerializableFunction { + @Override + public TableReference apply(String from) { + return parseTableSpec(from); + } + } + /** * A {@link PTransform} that reads from a BigQuery table and returns a * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table. @@ -345,6 +380,13 @@ public static class Read { * {@code "[dataset_id].[table_id]"} for tables within the current project. */ public static Bound from(String tableSpec) { + return new Bound().from(StaticValueProvider.of(tableSpec)); + } + + /** + * Same as {@code from(String)}, but with a {@link ValueProvider}. + */ + public static Bound from(ValueProvider tableSpec) { return new Bound().from(tableSpec); } @@ -352,6 +394,13 @@ public static Bound from(String tableSpec) { * Reads results received after executing the given query. */ public static Bound fromQuery(String query) { + return new Bound().fromQuery(StaticValueProvider.of(query)); + } + + /** + * Same as {@code from(String)}, but with a {@link ValueProvider}. + */ + public static Bound fromQuery(ValueProvider query) { return new Bound().fromQuery(query); } @@ -374,8 +423,8 @@ public static Bound withoutValidation() { * {@link PCollection} of {@link TableRow TableRows}. */ public static class Bound extends PTransform> { - @Nullable final String jsonTableRef; - @Nullable final String query; + @Nullable final ValueProvider jsonTableRef; + @Nullable final ValueProvider query; /** * Disable validation that the table exists or the query succeeds prior to pipeline @@ -403,7 +452,8 @@ private Bound() { } private Bound( - String name, @Nullable String query, @Nullable String jsonTableRef, boolean validate, + String name, @Nullable ValueProvider query, + @Nullable ValueProvider jsonTableRef, boolean validate, @Nullable Boolean flattenResults, @Nullable Boolean useLegacySql, @Nullable BigQueryServices bigQueryServices) { super(name); @@ -421,8 +471,14 @@ private Bound( * *

      Does not modify this object. */ - public Bound from(String tableSpec) { - return from(parseTableSpec(tableSpec)); + public Bound from(ValueProvider tableSpec) { + return new Bound( + name, query, + NestedValueProvider.of( + NestedValueProvider.of( + tableSpec, new TableSpecToTableRef()), + new TableRefToJson()), + validate, flattenResults, useLegacySql, bigQueryServices); } /** @@ -431,9 +487,7 @@ public Bound from(String tableSpec) { *

      Does not modify this object. */ public Bound from(TableReference table) { - return new Bound( - name, query, toJsonString(table), validate, flattenResults, useLegacySql, - bigQueryServices); + return from(StaticValueProvider.of(toTableSpec(table))); } /** @@ -450,6 +504,13 @@ name, query, toJsonString(table), validate, flattenResults, useLegacySql, * Standard SQL dialect, use {@link BigQueryIO.Read.Bound#usingStandardSql}. */ public Bound fromQuery(String query) { + return fromQuery(StaticValueProvider.of(query)); + } + + /** + * Like {@link #fromQuery(String)}, but from a {@link ValueProvider}. + */ + public Bound fromQuery(ValueProvider query) { return new Bound(name, query, jsonTableRef, validate, MoreObjects.firstNonNull(flattenResults, Boolean.TRUE), MoreObjects.firstNonNull(useLegacySql, Boolean.TRUE), @@ -520,7 +581,7 @@ public void validate(PBegin input) { } } - TableReference table = getTableWithDefaultProject(bqOptions); + ValueProvider table = getTableWithDefaultProject(bqOptions); checkState( table == null || query == null, @@ -547,22 +608,24 @@ public void validate(PBegin input) { // earlier stages of the pipeline or if a query depends on earlier stages of a pipeline. // For these cases the withoutValidation method can be used to disable the check. if (validate && table != null) { + checkState(table.isAccessible(), "Cannot call validate if table is dynamically set."); // Check for source table presence for early failure notification. DatasetService datasetService = getBigQueryServices().getDatasetService(bqOptions); - verifyDatasetPresence(datasetService, table); - verifyTablePresence(datasetService, table); + verifyDatasetPresence(datasetService, table.get()); + verifyTablePresence(datasetService, table.get()); } else if (validate && query != null) { + checkState(query.isAccessible(), "Cannot call validate if query is dynamically set."); JobService jobService = getBigQueryServices().getJobService(bqOptions); try { jobService.dryRunQuery( bqOptions.getProject(), new JobConfigurationQuery() - .setQuery(query) + .setQuery(query.get()) .setFlattenResults(flattenResults) .setUseLegacySql(useLegacySql)); } catch (Exception e) { throw new IllegalArgumentException( - String.format(QUERY_VALIDATION_FAILURE_ERROR, query), e); + String.format(QUERY_VALIDATION_FAILURE_ERROR, query.get()), e); } } } @@ -588,7 +651,7 @@ public PCollection apply(PBegin input) { } final String executingProject = bqOptions.getProject(); - if (!Strings.isNullOrEmpty(query)) { + if (query != null && (!query.isAccessible() || !Strings.isNullOrEmpty(query.get()))) { String queryTempDatasetId = "temp_dataset_" + uuid; String queryTempTableId = "temp_table_" + uuid; @@ -598,12 +661,15 @@ public PCollection apply(PBegin input) { .setTableId(queryTempTableId); source = BigQueryQuerySource.create( - jobIdToken, query, queryTempTableRef, flattenResults, useLegacySql, - extractDestinationDir, bqServices); + jobIdToken, query, NestedValueProvider.of( + StaticValueProvider.of( + toJsonString(queryTempTableRef)), new JsonTableRefToTableRef()), + flattenResults, useLegacySql, extractDestinationDir, bqServices); } else { - TableReference inputTable = getTableWithDefaultProject(bqOptions); + ValueProvider inputTable = getTableWithDefaultProject(bqOptions); source = BigQueryTableSource.create( - jobIdToken, inputTable, extractDestinationDir, bqServices, executingProject); + jobIdToken, inputTable, extractDestinationDir, bqServices, + StaticValueProvider.of(executingProject)); } PassThroughThenCleanup.CleanupOperation cleanupOperation = new PassThroughThenCleanup.CleanupOperation() { @@ -652,9 +718,11 @@ public void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("table", toTableSpec(table)) .withLabel("Table")); } - + String queryString = query == null + ? null : query.isAccessible() + ? query.get() : query.toString(); builder - .addIfNotNull(DisplayData.item("query", query) + .addIfNotNull(DisplayData.item("query", queryString) .withLabel("Query")) .addIfNotNull(DisplayData.item("flattenResults", flattenResults) .withLabel("Flatten Query Results")) @@ -670,28 +738,56 @@ public void populateDisplayData(DisplayData.Builder builder) { * *

      If the table's project is not specified, use the executing project. */ - @Nullable private TableReference getTableWithDefaultProject(BigQueryOptions bqOptions) { - TableReference table = getTable(); - if (table != null && Strings.isNullOrEmpty(table.getProjectId())) { + @Nullable private ValueProvider getTableWithDefaultProject( + BigQueryOptions bqOptions) { + ValueProvider table = getTableProvider(); + if (table == null) { + return table; + } + if (!table.isAccessible()) { + LOG.info("Using a dynamic value for table input. This must contain a project" + + " in the table reference: {}", table); + return table; + } + if (Strings.isNullOrEmpty(table.get().getProjectId())) { // If user does not specify a project we assume the table to be located in // the default project. - table.setProjectId(bqOptions.getProject()); + TableReference ref = table.get(); + ref.setProjectId(bqOptions.getProject()); + return NestedValueProvider.of(StaticValueProvider.of( + toJsonString(ref)), new JsonTableRefToTableRef()); } return table; } + /** + * Returns the table to read, or {@code null} if reading from a query instead. + */ + @Nullable + public ValueProvider getTableProvider() { + return jsonTableRef == null + ? null : NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableRef()); + } /** * Returns the table to read, or {@code null} if reading from a query instead. */ @Nullable public TableReference getTable() { - return fromJsonString(jsonTableRef, TableReference.class); + ValueProvider provider = getTableProvider(); + return provider == null ? null : provider.get(); } /** * Returns the query to be read, or {@code null} if reading from a table instead. */ public String getQuery() { + return query == null ? null : query.get(); + } + + /** + * Returns the query to be read, or {@code null} if reading from a table instead. + */ + public ValueProvider getQueryProivder() { return query; } @@ -788,44 +884,46 @@ static class BigQueryTableSource extends BigQuerySourceBase { static BigQueryTableSource create( String jobIdToken, - TableReference table, + ValueProvider table, String extractDestinationDir, BigQueryServices bqServices, - String executingProject) { + ValueProvider executingProject) { return new BigQueryTableSource( jobIdToken, table, extractDestinationDir, bqServices, executingProject); } - private final String jsonTable; + private final ValueProvider jsonTable; private final AtomicReference tableSizeBytes; private BigQueryTableSource( String jobIdToken, - TableReference table, + ValueProvider table, String extractDestinationDir, BigQueryServices bqServices, - String executingProject) { + ValueProvider executingProject) { super(jobIdToken, extractDestinationDir, bqServices, executingProject); - this.jsonTable = toJsonString(checkNotNull(table, "table")); + this.jsonTable = NestedValueProvider.of(checkNotNull(table, "table"), new TableRefToJson()); this.tableSizeBytes = new AtomicReference<>(); } @Override protected TableReference getTableToExtract(BigQueryOptions bqOptions) throws IOException { - return JSON_FACTORY.fromString(jsonTable, TableReference.class); + checkState(jsonTable.isAccessible()); + return JSON_FACTORY.fromString(jsonTable.get(), TableReference.class); } @Override public BoundedReader createReader(PipelineOptions options) throws IOException { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - TableReference tableRef = JSON_FACTORY.fromString(jsonTable, TableReference.class); + checkState(jsonTable.isAccessible()); + TableReference tableRef = JSON_FACTORY.fromString(jsonTable.get(), TableReference.class); return new BigQueryReader(this, bqServices.getReaderFromTable(bqOptions, tableRef)); } @Override public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws Exception { if (tableSizeBytes.get() == null) { - TableReference table = JSON_FACTORY.fromString(jsonTable, TableReference.class); + TableReference table = JSON_FACTORY.fromString(jsonTable.get(), TableReference.class); Long numBytes = bqServices.getDatasetService(options.as(BigQueryOptions.class)) .getTable(table.getProjectId(), table.getDatasetId(), table.getTableId()) @@ -843,7 +941,8 @@ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("table", jsonTable)); + String table = jsonTable.isAccessible() ? jsonTable.get() : jsonTable.toString(); + builder.add(DisplayData.item("table", table)); } } @@ -855,8 +954,8 @@ static class BigQueryQuerySource extends BigQuerySourceBase { static BigQueryQuerySource create( String jobIdToken, - String query, - TableReference queryTempTableRef, + ValueProvider query, + ValueProvider queryTempTableRef, Boolean flattenResults, Boolean useLegacySql, String extractDestinationDir, @@ -871,24 +970,26 @@ static BigQueryQuerySource create( bqServices); } - private final String query; - private final String jsonQueryTempTable; + private final ValueProvider query; + private final ValueProvider jsonQueryTempTable; private final Boolean flattenResults; private final Boolean useLegacySql; private transient AtomicReference dryRunJobStats; private BigQueryQuerySource( String jobIdToken, - String query, - TableReference queryTempTableRef, + ValueProvider query, + ValueProvider queryTempTableRef, Boolean flattenResults, Boolean useLegacySql, String extractDestinationDir, BigQueryServices bqServices) { super(jobIdToken, extractDestinationDir, bqServices, - checkNotNull(queryTempTableRef, "queryTempTableRef").getProjectId()); + NestedValueProvider.of( + checkNotNull(queryTempTableRef, "queryTempTableRef"), new TableRefToProjectId())); this.query = checkNotNull(query, "query"); - this.jsonQueryTempTable = toJsonString(queryTempTableRef); + this.jsonQueryTempTable = NestedValueProvider.of( + queryTempTableRef, new TableRefToJson()); this.flattenResults = checkNotNull(flattenResults, "flattenResults"); this.useLegacySql = checkNotNull(useLegacySql, "useLegacySql"); this.dryRunJobStats = new AtomicReference<>(); @@ -904,7 +1005,7 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { public BoundedReader createReader(PipelineOptions options) throws IOException { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); return new BigQueryReader(this, bqServices.getReaderFromQuery( - bqOptions, query, executingProject, flattenResults, useLegacySql)); + bqOptions, query.get(), executingProject.get(), flattenResults, useLegacySql)); } @Override @@ -925,7 +1026,7 @@ protected TableReference getTableToExtract(BigQueryOptions bqOptions) // 2. Create the temporary dataset in the query location. TableReference tableToExtract = - JSON_FACTORY.fromString(jsonQueryTempTable, TableReference.class); + JSON_FACTORY.fromString(jsonQueryTempTable.get(), TableReference.class); tableService.createDataset( tableToExtract.getProjectId(), tableToExtract.getDatasetId(), @@ -935,7 +1036,7 @@ protected TableReference getTableToExtract(BigQueryOptions bqOptions) // 3. Execute the query. String queryJobId = jobIdToken + "-query"; executeQuery( - executingProject, + executingProject.get(), queryJobId, tableToExtract, bqServices.getJobService(bqOptions)); @@ -944,8 +1045,9 @@ protected TableReference getTableToExtract(BigQueryOptions bqOptions) @Override protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { + checkState(jsonQueryTempTable.isAccessible()); TableReference tableToRemove = - JSON_FACTORY.fromString(jsonQueryTempTable, TableReference.class); + JSON_FACTORY.fromString(jsonQueryTempTable.get(), TableReference.class); DatasetService tableService = bqServices.getDatasetService(bqOptions); tableService.deleteTable( @@ -958,14 +1060,14 @@ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("query", query)); + builder.add(DisplayData.item("query", query.get())); } private synchronized JobStatistics dryRunQueryIfNeeded(BigQueryOptions bqOptions) throws InterruptedException, IOException { if (dryRunJobStats.get() == null) { JobStatistics jobStats = bqServices.getJobService(bqOptions).dryRunQuery( - executingProject, createBasicQueryConfig()); + executingProject.get(), createBasicQueryConfig()); dryRunJobStats.compareAndSet(null, jobStats); } return dryRunJobStats.get(); @@ -996,7 +1098,7 @@ private void executeQuery( private JobConfigurationQuery createBasicQueryConfig() { return new JobConfigurationQuery() - .setQuery(query) + .setQuery(query.get()) .setFlattenResults(flattenResults) .setUseLegacySql(useLegacySql); } @@ -1033,13 +1135,13 @@ private abstract static class BigQuerySourceBase extends BoundedSource protected final String jobIdToken; protected final String extractDestinationDir; protected final BigQueryServices bqServices; - protected final String executingProject; + protected final ValueProvider executingProject; private BigQuerySourceBase( String jobIdToken, String extractDestinationDir, BigQueryServices bqServices, - String executingProject) { + ValueProvider executingProject) { this.jobIdToken = checkNotNull(jobIdToken, "jobIdToken"); this.extractDestinationDir = checkNotNull(extractDestinationDir, "extractDestinationDir"); this.bqServices = checkNotNull(bqServices, "bqServices"); @@ -1087,7 +1189,7 @@ private List executeExtract( String jobId, TableReference table, JobService jobService) throws InterruptedException, IOException { JobReference jobRef = new JobReference() - .setProjectId(executingProject) + .setProjectId(executingProject.get()) .setJobId(jobId); String destinationUri = getExtractDestinationUri(extractDestinationDir); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java index 1f7b292fd975..0abf01d4bc18 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java @@ -91,6 +91,8 @@ public void testOurApiSurface() throws Exception { inPackage("com.google.protobuf"), inPackage("com.google.type"), inPackage("com.fasterxml.jackson.annotation"), + inPackage("com.fasterxml.jackson.core"), + inPackage("com.fasterxml.jackson.databind"), inPackage("io.grpc"), inPackage("java"), inPackage("javax"), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 40965e497aee..25caf63f18ca 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -111,6 +111,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -626,7 +627,7 @@ private void checkReadTableObjectWithValidate( private void checkReadQueryObjectWithValidate( BigQueryIO.Read.Bound bound, String query, boolean validate) { assertNull(bound.getTable()); - assertEquals(query, bound.query); + assertEquals(query, bound.getQuery()); assertEquals(validate, bound.getValidate()); } @@ -707,27 +708,27 @@ public void testBuildSourceWithTableReference() { public void testValidateReadSetsDefaultProject() throws Exception { String projectId = "someproject"; String datasetId = "somedataset"; + String tableId = "sometable"; BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject(projectId); bqOptions.setTempLocation("gs://testbucket/testdir"); + FakeDatasetService fakeDatasetService = + new FakeDatasetService().withTable(projectId, datasetId, tableId, null); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(mockJobService) - .withDatasetService(mockDatasetService); - when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow( - new RuntimeException("Unable to confirm BigQuery dataset presence")); + .withJobService(new FakeJobService()) + .withDatasetService(fakeDatasetService); Pipeline p = TestPipeline.create(bqOptions); TableReference tableRef = new TableReference(); tableRef.setDatasetId(datasetId); - tableRef.setTableId("sometable"); + tableRef.setTableId(tableId); thrown.expect(RuntimeException.class); // Message will be one of following depending on the execution environment. - thrown.expectMessage( - Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence")) - .or(Matchers.containsString("BigQuery dataset not found for table"))); + thrown.expectMessage(Matchers.containsString("Unsupported")); p.apply(BigQueryIO.Read.from(tableRef) .withTestServices(fakeBqServices)); } @@ -761,7 +762,7 @@ public void testBuildSourceWithTableAndQuery() { p.apply("ReadMyTable", BigQueryIO.Read .from("foo.com:project:somedataset.sometable") - .fromQuery("query")); + .fromQuery("query")); p.run(); } @@ -1371,7 +1372,8 @@ public void testBigQueryTableSourceThroughJsonAPI() throws Exception { TableReference table = BigQueryIO.parseTableSpec("project.data_set.table_name"); String extractDestinationDir = "mock://tempLocation"; BoundedSource bqSource = BigQueryTableSource.create( - jobIdToken, table, extractDestinationDir, fakeBqServices, "project"); + jobIdToken, StaticValueProvider.of(table), extractDestinationDir, fakeBqServices, + StaticValueProvider.of("project")); List expected = ImmutableList.of( new TableRow().set("name", "a").set("number", "1"), @@ -1408,7 +1410,8 @@ public void testBigQueryTableSourceInitSplit() throws Exception { TableReference table = BigQueryIO.parseTableSpec("project:data_set.table_name"); String extractDestinationDir = "mock://tempLocation"; BoundedSource bqSource = BigQueryTableSource.create( - jobIdToken, table, extractDestinationDir, fakeBqServices, "project"); + jobIdToken, StaticValueProvider.of(table), + extractDestinationDir, fakeBqServices, StaticValueProvider.of("project")); List expected = ImmutableList.of( new TableRow().set("name", "a").set("number", "1"), @@ -1473,7 +1476,8 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { String extractDestinationDir = "mock://tempLocation"; TableReference destinationTable = BigQueryIO.parseTableSpec("project:data_set.table_name"); BoundedSource bqSource = BigQueryQuerySource.create( - jobIdToken, "query", destinationTable, true /* flattenResults */, true /* useLegacySql */, + jobIdToken, StaticValueProvider.of("query"), StaticValueProvider.of(destinationTable), + true /* flattenResults */, true /* useLegacySql */, extractDestinationDir, fakeBqServices); List expected = ImmutableList.of( @@ -1564,7 +1568,8 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { String extractDestinationDir = "mock://tempLocation"; TableReference destinationTable = BigQueryIO.parseTableSpec("project:data_set.table_name"); BoundedSource bqSource = BigQueryQuerySource.create( - jobIdToken, "query", destinationTable, true /* flattenResults */, true /* useLegacySql */, + jobIdToken, StaticValueProvider.of("query"), StaticValueProvider.of(destinationTable), + true /* flattenResults */, true /* useLegacySql */, extractDestinationDir, fakeBqServices); List expected = ImmutableList.of( From 869b2710efdb90bc3ce5b6e9d4f3b49a3a804a63 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 7 Dec 2016 13:28:13 +0800 Subject: [PATCH 099/279] [FLINK-1102] Fix Aggregator Registration in Flink Batch Runner --- .../functions/FlinkProcessContextBase.java | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java index 42607dd0b910..6afca38dcce4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.collect.Iterables; -import java.io.Serializable; import java.util.Collection; import java.util.Collections; import java.util.Iterator; @@ -39,7 +38,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.RuntimeContext; import org.joda.time.Instant; @@ -256,15 +254,14 @@ protected abstract void outputWithTimestampAndWindow( @Override protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = - new SerializableFnAggregatorWrapper<>(combiner); - Accumulator existingAccum = - (Accumulator) runtimeContext.getAccumulator(name); - if (existingAccum != null) { - return wrapper; - } else { - runtimeContext.addAccumulator(name, wrapper); + @SuppressWarnings("unchecked") + SerializableFnAggregatorWrapper result = + (SerializableFnAggregatorWrapper) + runtimeContext.getAccumulator(name); + + if (result == null) { + result = new SerializableFnAggregatorWrapper<>(combiner); + runtimeContext.addAccumulator(name, result); } - return wrapper; - } + return result; } } From baf5e6bd9b1011f4c5c3974aa46393471b340c15 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 7 Dec 2016 08:37:33 +0100 Subject: [PATCH 100/279] [BEAM-1094] Set test scope for Kafka IO and junit --- runners/spark/pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index e34af15cc448..9a3adf6af93a 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -268,18 +268,20 @@ org.apache.beam beam-sdks-java-io-kafka + test org.apache.kafka kafka-clients 0.9.0.1 + test junit junit - provided + test hamcrest-core From dce3a196a3a26fdd42225520faf3d9084ee48123 Mon Sep 17 00:00:00 2001 From: Sela Date: Wed, 7 Dec 2016 11:20:07 +0200 Subject: [PATCH 101/279] [BEAM-329] Update Spark runner README. --- runners/spark/README.md | 59 ++++++------------- .../runners/spark/examples/WordCount.java | 5 +- 2 files changed, 21 insertions(+), 43 deletions(-) diff --git a/runners/spark/README.md b/runners/spark/README.md index ef42fa707909..aad65b3a914a 100644 --- a/runners/spark/README.md +++ b/runners/spark/README.md @@ -38,32 +38,25 @@ with Apache Spark. This runner allows to execute both batch and streaming pipeli - Side inputs/outputs - Encoding -### Sources and Sinks - -- Text -- Hadoop -- Avro -- Kafka - ### Fault-Tolerance The Spark runner fault-tolerance guarantees the same guarantees as [Apache Spark](http://spark.apache.org/). ### Monitoring -The Spark runner supports monitoring via Beam Aggregators implemented on top of Spark's [Accumulators](http://spark.apache.org/docs/latest/programming-guide.html#accumulators). -Spark also provides a web UI for monitoring, more details [here](http://spark.apache.org/docs/latest/monitoring.html). +The Spark runner supports user-defined counters via Beam Aggregators implemented on top of Spark's [Accumulators](http://spark.apache.org/docs/1.6.3/programming-guide.html#accumulators). +The Aggregators (defined by the pipeline author) and Spark's internal metrics are reported using Spark's [metrics system](http://spark.apache.org/docs/1.6.3/monitoring.html#metrics). +Spark also provides a web UI for monitoring, more details [here](http://spark.apache.org/docs/1.6.3/monitoring.html). ## Beam Model support ### Batch -The Spark runner provides support for batch processing of Beam bounded PCollections as Spark [RDD](http://spark.apache.org/docs/latest/programming-guide.html#resilient-distributed-datasets-rdds)s. +The Spark runner provides full support for the Beam Model in batch processing via Spark [RDD](http://spark.apache.org/docs/1.6.3/programming-guide.html#resilient-distributed-datasets-rdds)s. ### Streaming -The Spark runner currently provides partial support for stream processing of Beam unbounded PCollections as Spark [DStream](http://spark.apache.org/docs/latest/streaming-programming-guide.html#discretized-streams-dstreams)s. -Currently, both *FixedWindows* and *SlidingWindows* are supported, but only with processing-time triggers and discarding pane. +Providing full support for the Beam Model in streaming pipelines is under development. To follow-up you can subscribe to our [mailing list](http://beam.incubator.apache.org/get-started/support/). ### issue tracking @@ -84,19 +77,21 @@ Then switch to the newly created directory and run Maven to build the Apache Bea Now Apache Beam and the Spark Runner are installed in your local maven repository. -If we wanted to run a Beam pipeline with the default options of a single threaded Spark -instance in local mode, we would do the following: +If we wanted to run a Beam pipeline with the default options of a Spark instance in local mode, +we would do the following: Pipeline p = - EvaluationResult result = SparkRunner.create().run(p); + PipelineResult result = p.run(); + result.waitUntilFinish(); To create a pipeline runner to run against a different Spark cluster, with a custom master url we would do the following: - Pipeline p = - SparkPipelineOptions options = SparkPipelineOptionsFactory.create(); + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); options.setSparkMaster("spark://host:port"); - EvaluationResult result = SparkRunner.create(options).run(p); + Pipeline p = + PipelineResult result = p.run(); + result.waitUntilFinish(); ## Word Count Example @@ -108,12 +103,11 @@ Switch to the Spark runner directory: cd runners/spark -Then run the [word count example][wc] from the SDK using a single threaded Spark instance -in local mode: +Then run the [word count example][wc] from the SDK using a Spark instance in local mode: - mvn exec:exec -DmainClass=org.apache.beam.examples.WordCount \ - -Dinput=/tmp/kinglear.txt -Doutput=/tmp/out -Drunner=SparkRunner \ - -DsparkMaster=local + mvn exec:exec -DmainClass=org.apache.beam.runners.spark.examples.WordCount \ + -Dinput=/tmp/kinglear.txt -Doutput=/tmp/out -Drunner=SparkRunner \ + -DsparkMaster=local Check the output by running: @@ -122,24 +116,9 @@ Check the output by running: __Note: running examples using `mvn exec:exec` only works for Spark local mode at the moment. See the next section for how to run on a cluster.__ -[wc]: https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +[wc]: https://github.com/apache/incubator-beam/blob/master/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java ## Running on a Cluster Spark Beam pipelines can be run on a cluster using the `spark-submit` command. -First copy a text document to HDFS: - - curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt | hadoop fs -put - kinglear.txt - -Then run the word count example using Spark submit with the `yarn-client` master -(`yarn-cluster` works just as well): - - spark-submit \ - --class org.apache.beam.examples.WordCount \ - --master yarn-client \ - target/spark-runner-*-spark-app.jar \ - --inputFile=kinglear.txt --output=out --runner=SparkRunner --sparkMaster=yarn-client - -Check the output by running: - - hadoop fs -tail out-00000-of-00002 +TBD pending native HDFS support (currently blocked by [BEAM-59](https://issues.apache.org/jira/browse/BEAM-59)). diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java index 38dae38b92be..d7e520724608 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.values.PCollection; /** - * Duplicated to avoid dependency on beam-examples. + * Duplicated from beam-examples-java to avoid dependency. */ public class WordCount { @@ -126,8 +126,7 @@ public static void main(String[] args) { // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the // static FormatAsTextFn() to the ParDo transform. - //TODO: remove withoutValidation once possible - p.apply("ReadLines", TextIO.Read.from(options.getInputFile()).withoutValidation()) + p.apply("ReadLines", TextIO.Read.from(options.getInputFile())) .apply(new CountWords()) .apply(MapElements.via(new FormatAsTextFn())) .apply("WriteCounts", TextIO.Write.to(options.getOutput())); From b2d72237b592e1dcb5cca30f5cbc9a11d2890c0f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 15:20:28 -0800 Subject: [PATCH 102/279] Port most of DoFnRunner Javadoc to new DoFn --- .../org/apache/beam/runners/core/DoFnRunner.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index aac8e8f103bc..501667ef0122 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -18,29 +18,29 @@ package org.apache.beam.runners.core; import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.transforms.OldDoFn.ProcessContext; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; /** - * An wrapper interface that represents the execution of a {@link OldDoFn}. + * An wrapper interface that represents the execution of a {@link DoFn}. */ public interface DoFnRunner { /** - * Prepares and calls {@link OldDoFn#startBundle}. + * Prepares and calls a {@link DoFn DoFn's} {@link DoFn.StartBundle @StartBundle} method. */ void startBundle(); /** - * Calls {@link OldDoFn#processElement} with a {@link ProcessContext} containing the current - * element. + * Calls a {@link DoFn DoFn's} {@link DoFn.ProcessElement @ProcessElement} method with a + * {@link DoFn.ProcessContext} containing the provided element. */ void processElement(WindowedValue elem); /** - * Calls {@link OldDoFn#finishBundle} and performs additional tasks, such as - * flushing in-memory states. + * Calls a {@link DoFn DoFn's} {@link DoFn.FinishBundle @FinishBundle} method and performs + * additional tasks, such as flushing in-memory states. */ void finishBundle(); From 8e1e46e73edf9cce376ed7bd194db00edc3e60b4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 21:01:37 -0800 Subject: [PATCH 103/279] Port ParDoTest from OldDoFn to new DoFn --- .../apache/beam/sdk/transforms/ParDoTest.java | 238 +++++++----------- 1 file changed, 91 insertions(+), 147 deletions(-) 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 593f304f9955..975507660080 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 @@ -111,74 +111,9 @@ public void processElement(ProcessContext c, BoundedWindow window) { + ":" + window.maxTimestamp().getMillis()); } } - - static class TestOldDoFn extends OldDoFn { - enum State { UNSTARTED, STARTED, PROCESSING, FINISHED } - State state = State.UNSTARTED; - - final List> sideInputViews = new ArrayList<>(); - final List> sideOutputTupleTags = new ArrayList<>(); - - public TestOldDoFn() { - } - - public TestOldDoFn(List> sideInputViews, - List> sideOutputTupleTags) { - this.sideInputViews.addAll(sideInputViews); - this.sideOutputTupleTags.addAll(sideOutputTupleTags); - } - - @Override - public void startBundle(Context c) { - // The Fn can be reused, but only if FinishBundle has been called. - assertThat(state, anyOf(equalTo(State.UNSTARTED), equalTo(State.FINISHED))); - state = State.STARTED; - outputToAll(c, "started"); - } - - @Override - public void processElement(ProcessContext c) { - assertThat(state, - anyOf(equalTo(State.STARTED), equalTo(State.PROCESSING))); - state = State.PROCESSING; - outputToAllWithSideInputs(c, "processing: " + c.element()); - } - - @Override - public void finishBundle(Context c) { - assertThat(state, - anyOf(equalTo(State.STARTED), equalTo(State.PROCESSING))); - state = State.FINISHED; - outputToAll(c, "finished"); - } - - private void outputToAll(Context c, String value) { - c.output(value); - for (TupleTag sideOutputTupleTag : sideOutputTupleTags) { - c.sideOutput(sideOutputTupleTag, - sideOutputTupleTag.getId() + ": " + value); - } - } - - private void outputToAllWithSideInputs(ProcessContext c, String value) { - if (!sideInputViews.isEmpty()) { - List sideInputValues = new ArrayList<>(); - for (PCollectionView sideInputView : sideInputViews) { - sideInputValues.add(c.sideInput(sideInputView)); - } - value += ": " + sideInputValues; - } - c.output(value); - for (TupleTag sideOutputTupleTag : sideOutputTupleTags) { - c.sideOutput(sideOutputTupleTag, - sideOutputTupleTag.getId() + ": " + value); - } - } - } - - static class TestNoOutputDoFn extends OldDoFn { - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception {} + static class TestNoOutputDoFn extends DoFn { + @ProcessElement + public void processElement(DoFn.ProcessContext c) throws Exception {} } static class TestDoFn extends DoFn { @@ -254,52 +189,52 @@ private void outputToAllWithSideInputs(ProcessContext c, String value) { } } - static class TestStartBatchErrorDoFn extends OldDoFn { - @Override + static class TestStartBatchErrorDoFn extends DoFn { + @StartBundle public void startBundle(Context c) { throw new RuntimeException("test error in initialize"); } - @Override + @ProcessElement public void processElement(ProcessContext c) { // This has to be here. } } - static class TestProcessElementErrorDoFn extends OldDoFn { - @Override + static class TestProcessElementErrorDoFn extends DoFn { + @ProcessElement public void processElement(ProcessContext c) { throw new RuntimeException("test error in process"); } } - static class TestFinishBatchErrorDoFn extends OldDoFn { - @Override + static class TestFinishBatchErrorDoFn extends DoFn { + @ProcessElement public void processElement(ProcessContext c) { // This has to be here. } - @Override + @FinishBundle public void finishBundle(Context c) { throw new RuntimeException("test error in finalize"); } } - private static class StrangelyNamedDoer extends OldDoFn { - @Override + private static class StrangelyNamedDoer extends DoFn { + @ProcessElement public void processElement(ProcessContext c) { } } - static class TestOutputTimestampDoFn extends OldDoFn { - @Override + static class TestOutputTimestampDoFn extends DoFn { + @ProcessElement public void processElement(ProcessContext c) { Integer value = c.element(); c.outputWithTimestamp(value, new Instant(value.longValue())); } } - static class TestShiftTimestampDoFn extends OldDoFn { + static class TestShiftTimestampDoFn extends DoFn { private Duration allowedTimestampSkew; private Duration durationToShift; @@ -313,7 +248,7 @@ public TestShiftTimestampDoFn(Duration allowedTimestampSkew, public Duration getAllowedTimestampSkew() { return allowedTimestampSkew; } - @Override + @ProcessElement public void processElement(ProcessContext c) { Instant timestamp = c.timestamp(); checkNotNull(timestamp); @@ -322,8 +257,8 @@ public void processElement(ProcessContext c) { } } - static class TestFormatTimestampDoFn extends OldDoFn { - @Override + static class TestFormatTimestampDoFn extends DoFn { + @ProcessElement public void processElement(ProcessContext c) { checkNotNull(c.timestamp()); c.output("processing: " + c.element() + ", timestamp: " + c.timestamp().getMillis()); @@ -343,14 +278,14 @@ public PCollectionTuple apply(PCollection input) { return PCollectionTuple.of(BY2, by2).and(BY3, by3); } - static class FilterFn extends OldDoFn { + static class FilterFn extends DoFn { private final int divisor; FilterFn(int divisor) { this.divisor = divisor; } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { if (c.element() % divisor == 0) { c.output(c.element()); @@ -368,7 +303,7 @@ public void testParDo() { PCollection output = pipeline .apply(Create.of(inputs)) - .apply(ParDo.of(new TestOldDoFn())); + .apply(ParDo.of(new TestDoFn())); PAssert.that(output) .satisfies(ParDoTest.HasExpectedOutput.forInput(inputs)); @@ -402,7 +337,7 @@ public void testParDoEmpty() { PCollection output = pipeline .apply(Create.of(inputs).withCoder(VarIntCoder.of())) - .apply("TestOldDoFn", ParDo.of(new TestOldDoFn())); + .apply("TestDoFn", ParDo.of(new TestDoFn())); PAssert.that(output) .satisfies(ParDoTest.HasExpectedOutput.forInput(inputs)); @@ -420,7 +355,7 @@ public void testParDoEmptyOutputs() { PCollection output = pipeline .apply(Create.of(inputs).withCoder(VarIntCoder.of())) - .apply("TestOldDoFn", ParDo.of(new TestNoOutputDoFn())); + .apply("TestDoFn", ParDo.of(new TestNoOutputDoFn())); PAssert.that(output).empty(); @@ -443,7 +378,7 @@ public void testParDoWithSideOutputs() { PCollectionTuple outputs = pipeline .apply(Create.of(inputs)) .apply(ParDo - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.>asList(), Arrays.asList(sideOutputTag1, sideOutputTag2, sideOutputTag3))) .withOutputTags( @@ -486,7 +421,7 @@ public void testParDoEmptyWithSideOutputs() { PCollectionTuple outputs = pipeline .apply(Create.of(inputs)) .apply(ParDo - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.>asList(), Arrays.asList(sideOutputTag1, sideOutputTag2, sideOutputTag3))) .withOutputTags( @@ -552,8 +487,8 @@ public void testParDoWithOnlySideOutputs() { PCollectionTuple outputs = pipeline .apply(Create.of(inputs)) .apply(ParDo.withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) - .of(new OldDoFn(){ - @Override + .of(new DoFn(){ + @ProcessElement public void processElement(ProcessContext c) { c.sideOutput(sideOutputTag, c.element()); }})); @@ -575,7 +510,7 @@ public void testParDoWritingToUndeclaredSideOutput() { PCollection output = pipeline .apply(Create.of(inputs)) - .apply(ParDo.of(new TestOldDoFn( + .apply(ParDo.of(new TestDoFn( Arrays.>asList(), Arrays.asList(sideTag)))); @@ -594,8 +529,8 @@ public void testParDoUndeclaredSideOutputLimit() { // Success for a total of 1000 outputs. input - .apply("Success1000", ParDo.of(new OldDoFn() { - @Override + .apply("Success1000", ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { TupleTag specialSideTag = new TupleTag(){}; c.sideOutput(specialSideTag, "side"); @@ -610,8 +545,8 @@ public void processElement(ProcessContext c) { // Failure for a total of 1001 outputs. input - .apply("Failure1001", ParDo.of(new OldDoFn() { - @Override + .apply("Failure1001", ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { for (int i = 0; i < 1000; i++) { c.sideOutput(new TupleTag(){}, "side"); @@ -643,7 +578,7 @@ public void testParDoWithSideInputs() { PCollection output = pipeline .apply(Create.of(inputs)) .apply(ParDo.withSideInputs(sideInput1, sideInputUnread, sideInput2) - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.asList(sideInput1, sideInput2), Arrays.>asList()))); @@ -677,7 +612,7 @@ public void testParDoWithSideInputsIsCumulative() { .apply(ParDo.withSideInputs(sideInput1) .withSideInputs(sideInputUnread) .withSideInputs(sideInput2) - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.asList(sideInput1, sideInput2), Arrays.>asList()))); @@ -715,7 +650,7 @@ public void testMultiOutputParDoWithSideInputs() { .withSideInputs(sideInputUnread) .withSideInputs(sideInput2) .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.asList(sideInput1, sideInput2), Arrays.>asList()))); @@ -753,7 +688,7 @@ public void testMultiOutputParDoWithSideInputsIsCumulative() { .withSideInputs(sideInputUnread) .withSideInputs(sideInput2) .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.asList(sideInput1, sideInput2), Arrays.>asList()))); @@ -777,7 +712,7 @@ public void testParDoReadingFromUnknownSideInput() { .apply(View.asSingleton()); pipeline.apply("CreateMain", Create.of(inputs)) - .apply(ParDo.of(new TestOldDoFn( + .apply(ParDo.of(new TestDoFn( Arrays.>asList(sideView), Arrays.>asList()))); @@ -834,7 +769,7 @@ public void testParDoWithErrorInFinishBatch() { @Test public void testParDoOutputNameBasedOnDoFnWithTrimmedSuffix() { Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of(1)).apply(ParDo.of(new TestOldDoFn())); + PCollection output = p.apply(Create.of(1)).apply(ParDo.of(new TestDoFn())); assertThat(output.getName(), containsString("ParDo(Test)")); } @@ -842,7 +777,7 @@ public void testParDoOutputNameBasedOnDoFnWithTrimmedSuffix() { public void testParDoOutputNameBasedOnLabel() { Pipeline p = TestPipeline.create(); PCollection output = - p.apply(Create.of(1)).apply("MyParDo", ParDo.of(new TestOldDoFn())); + p.apply(Create.of(1)).apply("MyParDo", ParDo.of(new TestDoFn())); assertThat(output.getName(), containsString("MyParDo")); } @@ -878,7 +813,7 @@ public void testParDoWithSideOutputsName() { PCollectionTuple outputs = p .apply(Create.of(Arrays.asList(3, -42, 666))).setName("MyInput") .apply("MyParDo", ParDo - .of(new TestOldDoFn( + .of(new TestDoFn( Arrays.>asList(), Arrays.asList(sideOutputTag1, sideOutputTag2, sideOutputTag3))) .withOutputTags( @@ -906,7 +841,7 @@ public void testParDoInCustomTransform() { .apply("CustomTransform", new PTransform, PCollection>() { @Override public PCollection apply(PCollection input) { - return input.apply(ParDo.of(new TestOldDoFn())); + return input.apply(ParDo.of(new TestDoFn())); } }); @@ -943,8 +878,8 @@ public void testMultiOutputChaining() { @Test public void testJsonEscaping() { // Declare an arbitrary function and make sure we can serialize it - OldDoFn doFn = new OldDoFn() { - @Override + DoFn doFn = new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element() + 1); } @@ -996,24 +931,25 @@ public void registerByteSizeObserver( } } - private static class SideOutputDummyFn extends OldDoFn { + private static class SideOutputDummyFn extends DoFn { private TupleTag sideTag; public SideOutputDummyFn(TupleTag sideTag) { this.sideTag = sideTag; } - @Override + + @ProcessElement public void processElement(ProcessContext c) { c.output(1); c.sideOutput(sideTag, new TestDummy()); } } - private static class MainOutputDummyFn extends OldDoFn { + private static class MainOutputDummyFn extends DoFn { private TupleTag sideTag; public MainOutputDummyFn(TupleTag sideTag) { this.sideTag = sideTag; } - @Override + @ProcessElement public void processElement(ProcessContext c) { c.output(new TestDummy()); c.sideOutput(sideTag, 1); @@ -1190,8 +1126,9 @@ public void testMainOutputApplySideOutputNoCoder() { .apply(ParDo .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) .of( - new OldDoFn() { - @Override public void processElement(ProcessContext context) { + new DoFn() { + @ProcessElement + public void processElement(ProcessContext context) { TestDummy element = context.element(); context.output(element); context.sideOutput(sideOutputTag, element); @@ -1204,8 +1141,9 @@ public void testMainOutputApplySideOutputNoCoder() { // on a missing coder. tuple.get(mainOutputTag) .setCoder(TestDummyCoder.of()) - .apply("Output1", ParDo.of(new OldDoFn() { - @Override public void processElement(ProcessContext context) { + .apply("Output1", ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext context) { context.output(1); } })); @@ -1251,8 +1189,8 @@ public void testParDoSideOutputWithTimestamp() { PCollection output = input .apply(ParDo.withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)).of( - new OldDoFn() { - @Override + new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { c.sideOutputWithTimestamp( sideOutputTag, c.element(), new Instant(c.element().longValue())); @@ -1369,29 +1307,33 @@ public Void apply(Iterable input) { public void testWindowingInStartAndFinishBundle() { Pipeline pipeline = TestPipeline.create(); - PCollection output = pipeline - .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) - .apply(Window.into(FixedWindows.of(Duration.millis(1)))) - .apply(ParDo.of(new OldDoFn() { - @Override - public void startBundle(Context c) { - c.outputWithTimestamp("start", new Instant(2)); - System.out.println("Start: 2"); - } - - @Override - public void processElement(ProcessContext c) { - c.output(c.element()); - System.out.println("Process: " + c.element() + ":" + c.timestamp().getMillis()); - } - - @Override - public void finishBundle(Context c) { - c.outputWithTimestamp("finish", new Instant(3)); - System.out.println("Finish: 3"); - } - })) - .apply(ParDo.of(new PrintingDoFn())); + PCollection output = + pipeline + .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) + .apply(Window.into(FixedWindows.of(Duration.millis(1)))) + .apply( + ParDo.of( + new DoFn() { + @StartBundle + public void startBundle(Context c) { + c.outputWithTimestamp("start", new Instant(2)); + System.out.println("Start: 2"); + } + + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.element()); + System.out.println( + "Process: " + c.element() + ":" + c.timestamp().getMillis()); + } + + @FinishBundle + public void finishBundle(Context c) { + c.outputWithTimestamp("finish", new Instant(3)); + System.out.println("Finish: 3"); + } + })) + .apply(ParDo.of(new PrintingDoFn())); PAssert.that(output).satisfies(new Checker()); @@ -1406,13 +1348,15 @@ public void testWindowingInStartBundleException() { pipeline .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) .apply(Window.into(FixedWindows.of(Duration.millis(1)))) - .apply(ParDo.of(new OldDoFn() { - @Override + .apply( + ParDo.of( + new DoFn() { + @StartBundle public void startBundle(Context c) { c.output("start"); } - @Override + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element()); } @@ -1423,8 +1367,8 @@ public void processElement(ProcessContext c) { } @Test public void testDoFnDisplayData() { - OldDoFn fn = new OldDoFn() { - @Override + DoFn fn = new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { } From 55d333bff68809ff1a9154491ace02d2d16e3b85 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 5 Dec 2016 14:29:05 -0800 Subject: [PATCH 104/279] Only provide expanded Inputs and Outputs This removes PInput and POutput from the immediate API Surface of TransformHierarchy.Node, and forces Pipeline Visitors to access only the expanded version of the output. This is part of the move towards the runner-agnostic representation of a graph. --- .../apex/translation/TranslationContext.java | 4 +-- .../runners/direct/DirectGraphVisitor.java | 9 +++---- .../direct/KeyedPValueTrackingVisitor.java | 2 +- .../FlinkBatchPipelineTranslator.java | 4 +-- .../FlinkStreamingPipelineTranslator.java | 7 ++--- .../dataflow/DataflowPipelineTranslator.java | 3 +-- .../beam/runners/spark/SparkRunner.java | 17 ++++++------ .../beam/sdk/runners/TransformHierarchy.java | 26 ++++++++++--------- .../sdk/runners/TransformHierarchyTest.java | 13 ++++------ 9 files changed, 38 insertions(+), 47 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index 259afbdd7092..3bf01a87dc4d 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.PCollection; @@ -72,8 +71,7 @@ public InputT getViewInput(PCollectionView view) { } public void setCurrentTransform(TransformHierarchy.Node treeNode) { - this.currentTransform = AppliedPTransform.of(treeNode.getFullName(), - treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform()); + this.currentTransform = treeNode.toAppliedPTransform(); } public ApexPipelineOptions getPipelineOptions() { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index cd9d1201dac6..4f38bce1e9d9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -79,13 +79,13 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { - toFinalize.removeAll(node.getInput().expand()); + toFinalize.removeAll(node.getInputs()); AppliedPTransform appliedTransform = getAppliedTransform(node); stepNames.put(appliedTransform, genStepName()); - if (node.getInput().expand().isEmpty()) { + if (node.getInputs().isEmpty()) { rootTransforms.add(appliedTransform); } else { - for (PValue value : node.getInput().expand()) { + for (PValue value : node.getInputs()) { primitiveConsumers.put(value, appliedTransform); } } @@ -111,8 +111,7 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { @SuppressWarnings({"rawtypes", "unchecked"}) - AppliedPTransform application = AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform()); + AppliedPTransform application = node.toAppliedPTransform(); return application; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 5dc24c2b8581..4161f9e25def 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -74,7 +74,7 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { if (node.isRootNode()) { finalized = true; } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) { - keyedValues.addAll(node.getOutput().expand()); + keyedValues.addAll(node.getOutputs()); } } 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 805c41cd7916..209be69c5e48 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 @@ -20,7 +20,6 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -113,8 +112,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { BatchTransformTranslator typedTranslator = (BatchTransformTranslator) translator; // create the applied PTransform on the batchContext - batchContext.setCurrentTransform(AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + batchContext.setCurrentTransform(node.toAppliedPTransform()); typedTranslator.translateNode(typedTransform, batchContext); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java index a07dc3d7bfe2..23f4d34da460 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -109,8 +108,7 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; // create the applied PTransform on the streamingContext - streamingContext.setCurrentTransform(AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + streamingContext.setCurrentTransform(node.toAppliedPTransform()); typedTranslator.translateNode(typedTransform, streamingContext); } @@ -125,8 +123,7 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { @SuppressWarnings("unchecked") StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; - streamingContext.setCurrentTransform(AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + streamingContext.setCurrentTransform(node.toAppliedPTransform()); return typedTranslator.canTranslate(typedTransform, streamingContext); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index f43e176d0b6d..87830565b7e1 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -531,8 +531,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { "no translator registered for " + transform); } LOG.debug("Translating {}", transform); - currentTransform = AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform); + currentTransform = node.toAppliedPTransform(); translator.translate(transform, this); currentTransform = null; } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index d51ee7d01960..3d98b87a351f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.spark; +import com.google.common.collect.Iterables; import java.util.Collection; import java.util.List; import java.util.concurrent.ExecutorService; @@ -42,7 +43,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; @@ -278,8 +278,11 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { } private boolean shouldDefer(TransformHierarchy.Node node) { - PInput input = node.getInput(); // if the input is not a PCollection, or it is but with non merging windows, don't defer. + if (node.getInputs().size() != 1) { + return false; + } + PValue input = Iterables.getOnlyElement(node.getInputs()); if (!(input instanceof PCollection) || ((PCollection) input).getWindowingStrategy().getWindowFn().isNonMerging()) { return false; @@ -319,8 +322,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { @SuppressWarnings("unchecked") TransformEvaluator evaluator = translate(node, transform, transformClass); LOG.info("Evaluating {}", transform); - AppliedPTransform appliedTransform = - AppliedPTransform.of(node.getFullName(), node.getInput(), node.getOutput(), transform); + AppliedPTransform appliedTransform = node.toAppliedPTransform(); ctxt.setCurrentTransform(appliedTransform); evaluator.evaluate(transform, ctxt); ctxt.setCurrentTransform(null); @@ -337,12 +339,11 @@ TransformEvaluator translate( // usually, the input determines if the PCollection to apply the next transformation to // is BOUNDED or UNBOUNDED, meaning RDD/DStream. Collection pValues; - PInput pInput = node.getInput(); - if (pInput instanceof PBegin) { + if (node.getInputs().isEmpty()) { // in case of a PBegin, it's the output. - pValues = node.getOutput().expand(); + pValues = node.getOutputs(); } else { - pValues = pInput.expand(); + pValues = node.getInputs(); } PCollection.IsBounded isNodeBounded = isBoundedCollection(pValues); // translate accordingly. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index e9829cc56099..33d5231ac24f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -83,8 +84,8 @@ public Node pushNode(String name, PInput input, PTransform transform) { */ public void finishSpecifyingInput() { // Inputs must be completely specified before they are consumed by a transform. - current.getInput().finishSpecifying(); - for (PValue inputValue : current.getInput().expand()) { + for (PValue inputValue : current.getInputs()) { + inputValue.finishSpecifying(); checkState(producers.get(inputValue) != null, "Producer unknown for input %s", inputValue); inputValue.finishSpecifying(); } @@ -101,6 +102,7 @@ public void finishSpecifyingInput() { * nodes. */ public void setOutput(POutput output) { + output.finishSpecifyingOutput(); for (PValue value : output.expand()) { if (!producers.containsKey(value)) { producers.put(value, current); @@ -253,11 +255,9 @@ public String getFullName() { return fullName; } - /** - * Returns the transform input, in unexpanded form. - */ - public PInput getInput() { - return input; + /** Returns the transform input, in unexpanded form. */ + public Collection getInputs() { + return input == null ? Collections.emptyList() : input.expand(); } /** @@ -296,13 +296,15 @@ private void setOutput(POutput output) { } /** Returns the transform output, in unexpanded form. */ - public POutput getOutput() { - return output; + public Collection getOutputs() { + return output == null ? Collections.emptyList() : output.expand(); } - AppliedPTransform toAppliedPTransform() { - return AppliedPTransform.of( - getFullName(), getInput(), getOutput(), (PTransform) getTransform()); + /** + * Returns the {@link AppliedPTransform} representing this {@link Node}. + */ + public AppliedPTransform toAppliedPTransform() { + return AppliedPTransform.of(getFullName(), input, output, (PTransform) getTransform()); } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index f4488f4f8cd3..ea4318828f4c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -20,7 +20,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; import java.util.HashSet; @@ -38,8 +37,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.hamcrest.Matchers; import org.junit.Before; @@ -172,24 +169,24 @@ public Long apply(Long input) { TransformHierarchy.Node compositeNode = hierarchy.pushNode("Create", begin, create); assertThat(hierarchy.getCurrent(), equalTo(compositeNode)); - assertThat(compositeNode.getInput(), Matchers.equalTo(begin)); + assertThat(compositeNode.getInputs(), Matchers.emptyIterable()); assertThat(compositeNode.getTransform(), Matchers.>equalTo(create)); // Not yet set - assertThat(compositeNode.getOutput(), nullValue()); + assertThat(compositeNode.getOutputs(), Matchers.emptyIterable()); assertThat(compositeNode.getEnclosingNode().isRootNode(), is(true)); TransformHierarchy.Node primitiveNode = hierarchy.pushNode("Create/Read", begin, read); assertThat(hierarchy.getCurrent(), equalTo(primitiveNode)); hierarchy.setOutput(created); hierarchy.popNode(); - assertThat(primitiveNode.getOutput(), Matchers.equalTo(created)); - assertThat(primitiveNode.getInput(), Matchers.equalTo(begin)); + assertThat(primitiveNode.getOutputs(), Matchers.containsInAnyOrder(created)); + assertThat(primitiveNode.getInputs(), Matchers.emptyIterable()); assertThat(primitiveNode.getTransform(), Matchers.>equalTo(read)); assertThat(primitiveNode.getEnclosingNode(), equalTo(compositeNode)); hierarchy.setOutput(created); // The composite is listed as outputting a PValue created by the contained primitive - assertThat(compositeNode.getOutput(), Matchers.equalTo(created)); + assertThat(compositeNode.getOutputs(), Matchers.containsInAnyOrder(created)); // The producer of that PValue is still the primitive in which it is first output assertThat(hierarchy.getProducer(created), equalTo(primitiveNode)); hierarchy.popNode(); From 43fef2775145f67def3ab8a246ecca192a7d650b Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 7 Dec 2016 20:06:57 +0800 Subject: [PATCH 105/279] [BEAM-905] Add shading config to examples archetype and enable it for Flink This makes the Flink quickstart work out of the box. --- .../resources/archetype-resources/pom.xml | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index df2e9f351268..95d163c27cc6 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -85,6 +85,38 @@ false + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + package + + shade + + + ${project.artifactId}-bundled-${project.version} + + + *:* + + META-INF/LICENSE + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + @@ -140,6 +172,14 @@ runtime + + + + org.apache.maven.plugins + maven-shade-plugin + + + From 6439f701d1008d6a0432828e11e0fcc8a4fe6ecc Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 8 Dec 2016 07:40:58 +0800 Subject: [PATCH 106/279] [BEAM-1108] DataflowRunner: remove deprecated TEARDOWN_POLICY control --- .../dataflow/DataflowPipelineTranslator.java | 4 -- .../DataflowPipelineWorkerPoolOptions.java | 45 ------------------- 2 files changed, 49 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 87830565b7e1..8048df9d519a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -424,10 +424,6 @@ public Job translate(List packages) { WorkerPool workerPool = new WorkerPool(); - if (options.getTeardownPolicy() != null) { - workerPool.setTeardownPolicy(options.getTeardownPolicy().getTeardownPolicyName()); - } - if (options.isStreaming()) { job.setType("JOB_TYPE_STREAMING"); } else { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index ffb5a3a9f687..157321a13e2a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -190,51 +190,6 @@ public String create(PipelineOptions options) { String getWorkerMachineType(); void setWorkerMachineType(String value); - /** - * The policy for tearing down the workers spun up by the service. - * - * @deprecated Dataflow Service will only support TEARDOWN_ALWAYS policy in the future. - */ - @Deprecated - enum TeardownPolicy { - /** - * All VMs created for a Dataflow job are deleted when the job finishes, regardless of whether - * it fails or succeeds. - */ - TEARDOWN_ALWAYS("TEARDOWN_ALWAYS"), - /** - * All VMs created for a Dataflow job are left running when the job finishes, regardless of - * whether it fails or succeeds. - */ - TEARDOWN_NEVER("TEARDOWN_NEVER"), - /** - * All VMs created for a Dataflow job are deleted when the job succeeds, but are left running - * when it fails. (This is typically used for debugging failing jobs by SSHing into the - * workers.) - */ - TEARDOWN_ON_SUCCESS("TEARDOWN_ON_SUCCESS"); - - private final String teardownPolicy; - - TeardownPolicy(String teardownPolicy) { - this.teardownPolicy = teardownPolicy; - } - - public String getTeardownPolicyName() { - return this.teardownPolicy; - } - } - - /** - * The teardown policy for the VMs. - * - *

      If unset, the Dataflow service will choose a reasonable default. - */ - @Description("The teardown policy for the VMs. If unset, the Dataflow service will " - + "choose a reasonable default.") - TeardownPolicy getTeardownPolicy(); - void setTeardownPolicy(TeardownPolicy value); - /** * List of local files to make available to workers. * From 1b1252074dd6b57f4fb88ceb82c704d3d3d8147f Mon Sep 17 00:00:00 2001 From: Alexey Diomin Date: Wed, 7 Dec 2016 09:39:27 +0400 Subject: [PATCH 107/279] [BEAM-1095] Add support set config for reuse-object on flink --- .../flink/FlinkPipelineExecutionEnvironment.java | 12 ++++++++++++ .../beam/runners/flink/FlinkPipelineOptions.java | 5 +++++ 2 files changed, 17 insertions(+) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index 391c3f286b09..69dcd5e86132 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -159,6 +159,12 @@ private ExecutionEnvironment createBatchExecutionEnvironment() { // set parallelism in the options (required by some execution code) options.setParallelism(flinkBatchEnv.getParallelism()); + if (options.getObjectReuse()) { + flinkBatchEnv.getConfig().enableObjectReuse(); + } else { + flinkBatchEnv.getConfig().disableObjectReuse(); + } + return flinkBatchEnv; } @@ -197,6 +203,12 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() { // set parallelism in the options (required by some execution code) options.setParallelism(flinkStreamEnv.getParallelism()); + if (options.getObjectReuse()) { + flinkStreamEnv.getConfig().enableObjectReuse(); + } else { + flinkStreamEnv.getConfig().disableObjectReuse(); + } + // default to event time flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index be99f291d6ae..3bb358e6b71e 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -83,6 +83,11 @@ public interface FlinkPipelineOptions Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); + @Description("Sets the behavior of reusing objects.") + @Default.Boolean(false) + Boolean getObjectReuse(); + void setObjectReuse(Boolean reuse); + /** * Sets a state backend to store Beam's state during computation. * Note: Only applicable when executing in streaming mode. From f1a5704a505b01d7d4649b61d1f6697859367964 Mon Sep 17 00:00:00 2001 From: Alexey Diomin Date: Wed, 7 Dec 2016 09:48:35 +0400 Subject: [PATCH 108/279] [BEAM-1096] Flink streaming side output optimization using SplitStream --- .../FlinkStreamingTransformTranslators.java | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) 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 47935eb3d6ae..7b32c7649a38 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 @@ -78,11 +78,13 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.KeyedStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.datastream.SplitStream; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; @@ -554,6 +556,14 @@ public void translateNode( .transform(transform.getName(), outputUnionTypeInformation, doFnOperator); } + SplitStream splitStream = unionOutputStream + .split(new OutputSelector() { + @Override + public Iterable select(RawUnionValue value) { + return Collections.singletonList(Integer.toString(value.getUnionTag())); + } + }); + for (Map.Entry, PCollection> output : outputs.entrySet()) { final int outputTag = tagsToLabels.get(output.getKey()); @@ -561,17 +571,15 @@ public void translateNode( context.getTypeInfo(output.getValue()); @SuppressWarnings("unchecked") - DataStream filtered = - unionOutputStream.flatMap(new FlatMapFunction() { - @Override - public void flatMap(RawUnionValue value, Collector out) throws Exception { - if (value.getUnionTag() == outputTag) { - out.collect(value.getValue()); - } - } - }).returns(outputTypeInfo); + DataStream unwrapped = splitStream.select(String.valueOf(outputTag)) + .flatMap(new FlatMapFunction() { + @Override + public void flatMap(RawUnionValue value, Collector out) throws Exception { + out.collect(value.getValue()); + } + }).returns(outputTypeInfo); - context.setOutputDataStream(output.getValue(), filtered); + context.setOutputDataStream(output.getValue(), unwrapped); } } From 3f8c80769a3bb38da64c6628fd8c4669fcac794b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:10:06 -0800 Subject: [PATCH 109/279] Add DoFn.OnTimerContext --- .../org/apache/beam/sdk/transforms/DoFn.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 7aabec980f1b..699403f5b1a1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.state.State; @@ -294,6 +295,27 @@ public abstract class ProcessContext extends Context { public abstract PaneInfo pane(); } + /** + * Information accessible when running a {@link DoFn.OnTimer} method. + */ + public abstract class OnTimerContext extends Context { + + /** + * Returns the timestamp of the current timer. + */ + public abstract Instant timestamp(); + + /** + * Returns the window in which the timer is firing. + */ + public abstract BoundedWindow window(); + + /** + * Returns the time domain of the current timer. + */ + public abstract TimeDomain timeDomain(); + } + /** * Returns the allowed timestamp skew duration, which is the maximum * duration that timestamps can be shifted backward in From 42b506f06dbd73e03a2cfad4e7677e9698b3c020 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:18:18 -0800 Subject: [PATCH 110/279] Add OnTimerContext parameter support to DoFnSignature --- .../reflect/ByteBuddyDoFnInvokerFactory.java | 6 ++ .../sdk/transforms/reflect/DoFnSignature.java | 26 +++++- .../transforms/reflect/DoFnSignatures.java | 90 +++++++++++++++---- .../DoFnSignaturesSplittableDoFnTest.java | 3 +- .../reflect/DoFnSignaturesTest.java | 47 ++++++++++ 5 files changed, 154 insertions(+), 18 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java index 8750d64b1f75..3480603fdaef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java @@ -69,6 +69,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.Cases; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ContextParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.InputProviderParameter; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.OnTimerContextParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.OutputReceiverParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ProcessContextParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.RestrictionTrackerParameter; @@ -553,6 +554,11 @@ public StackManipulation dispatch(ProcessContextParameter p) { PROCESS_CONTEXT_PARAMETER_METHOD, DoFn.class))); } + @Override + public StackManipulation dispatch(OnTimerContextParameter p) { + throw new UnsupportedOperationException("OnTimerContext is not yet supported."); + } + @Override public StackManipulation dispatch(WindowParameter p) { return new StackManipulation.Compound( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index 07509498da78..ccc9ac3e90cd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -175,6 +175,8 @@ public ResultT match(Cases cases) { return cases.dispatch((ContextParameter) this); } else if (this instanceof ProcessContextParameter) { return cases.dispatch((ProcessContextParameter) this); + } else if (this instanceof OnTimerContextParameter) { + return cases.dispatch((OnTimerContextParameter) this); } else if (this instanceof WindowParameter) { return cases.dispatch((WindowParameter) this); } else if (this instanceof RestrictionTrackerParameter) { @@ -200,6 +202,7 @@ public ResultT match(Cases cases) { public interface Cases { ResultT dispatch(ContextParameter p); ResultT dispatch(ProcessContextParameter p); + ResultT dispatch(OnTimerContextParameter p); ResultT dispatch(WindowParameter p); ResultT dispatch(InputProviderParameter p); ResultT dispatch(OutputReceiverParameter p); @@ -224,6 +227,11 @@ public ResultT dispatch(ProcessContextParameter p) { return dispatchDefault(p); } + @Override + public ResultT dispatch(OnTimerContextParameter p) { + return dispatchDefault(p); + } + @Override public ResultT dispatch(WindowParameter p) { return dispatchDefault(p); @@ -261,12 +269,14 @@ public ResultT dispatch(TimerParameter p) { new AutoValue_DoFnSignature_Parameter_ContextParameter(); private static final ProcessContextParameter PROCESS_CONTEXT_PARAMETER = new AutoValue_DoFnSignature_Parameter_ProcessContextParameter(); + private static final OnTimerContextParameter ON_TIMER_CONTEXT_PARAMETER = + new AutoValue_DoFnSignature_Parameter_OnTimerContextParameter(); private static final InputProviderParameter INPUT_PROVIDER_PARAMETER = new AutoValue_DoFnSignature_Parameter_InputProviderParameter(); private static final OutputReceiverParameter OUTPUT_RECEIVER_PARAMETER = new AutoValue_DoFnSignature_Parameter_OutputReceiverParameter(); - /** Returns a {@link ProcessContextParameter}. */ + /** Returns a {@link ContextParameter}. */ public static ContextParameter context() { return CONTEXT_PARAMETER; } @@ -276,6 +286,11 @@ public static ProcessContextParameter processContext() { return PROCESS_CONTEXT_PARAMETER; } + /** Returns a {@link OnTimerContextParameter}. */ + public static OnTimerContextParameter onTimerContext() { + return ON_TIMER_CONTEXT_PARAMETER; + } + /** Returns a {@link WindowParameter}. */ public static WindowParameter boundedWindow(TypeDescriptor windowT) { return new AutoValue_DoFnSignature_Parameter_WindowParameter(windowT); @@ -333,6 +348,15 @@ public abstract static class ProcessContextParameter extends Parameter { ProcessContextParameter() {} } + /** + * Descriptor for a {@link Parameter} of type {@link DoFn.OnTimerContext}. + * + *

      All such descriptors are equal. + */ + @AutoValue + public abstract static class OnTimerContextParameter extends Parameter { + OnTimerContextParameter() {} + } /** * Descriptor for a {@link Parameter} of type {@link BoundedWindow}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index 83d67b7f3eb0..e3ba9663bcaa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -22,6 +22,7 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -47,7 +48,6 @@ import org.apache.beam.sdk.transforms.DoFn.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter; -import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ProcessContextParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.RestrictionTrackerParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.StateParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimerParameter; @@ -74,6 +74,29 @@ public class DoFnSignatures { private static final Map, DoFnSignature> signatureCache = new LinkedHashMap<>(); + private static final Collection> + ALLOWED_NON_SPLITTABLE_PROCESS_ELEMENT_PARAMETERS = + ImmutableList.of( + Parameter.ProcessContextParameter.class, + Parameter.WindowParameter.class, + Parameter.TimerParameter.class, + Parameter.StateParameter.class, + Parameter.InputProviderParameter.class, + Parameter.OutputReceiverParameter.class); + + private static final Collection> + ALLOWED_SPLITTABLE_PROCESS_ELEMENT_PARAMETERS = + ImmutableList.of( + Parameter.ProcessContextParameter.class, Parameter.RestrictionTrackerParameter.class); + + private static final Collection> + ALLOWED_ON_TIMER_PARAMETERS = + ImmutableList.of( + Parameter.OnTimerContextParameter.class, + Parameter.WindowParameter.class, + Parameter.TimerParameter.class, + Parameter.StateParameter.class); + /** @return the {@link DoFnSignature} for the given {@link DoFn} instance. */ public static > DoFnSignature signatureForDoFn(FnT fn) { return getSignature(fn.getClass()); @@ -582,6 +605,18 @@ private static TypeDescriptor.Context> d .where(new TypeParameter() {}, outputT); } + /** + * Generates a {@link TypeDescriptor} for {@code DoFn.Context} given {@code + * InputT} and {@code OutputT}. + */ + private static + TypeDescriptor.OnTimerContext> doFnOnTimerContextTypeOf( + TypeDescriptor inputT, TypeDescriptor outputT) { + return new TypeDescriptor.OnTimerContext>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); + } + /** * Generates a {@link TypeDescriptor} for {@code DoFn.InputProvider} given {@code InputT}. */ @@ -621,7 +656,7 @@ static DoFnSignature.OnTimerMethod analyzeOnTimerMethod( List extraParameters = new ArrayList<>(); ErrorReporter onTimerErrors = errors.forMethod(DoFn.OnTimer.class, m); for (int i = 0; i < params.length; ++i) { - extraParameters.add( + Parameter parameter = analyzeExtraParameter( onTimerErrors, fnContext, @@ -633,7 +668,14 @@ static DoFnSignature.OnTimerMethod analyzeOnTimerMethod( fnClass.resolveType(params[i]), Arrays.asList(m.getParameterAnnotations()[i])), inputT, - outputT)); + outputT); + + checkParameterOneOf( + errors, + parameter, + ALLOWED_ON_TIMER_PARAMETERS); + + extraParameters.add(parameter); } return DoFnSignature.OnTimerMethod.create(m, timerId, windowT, extraParameters); @@ -679,20 +721,15 @@ static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( methodContext.addParameter(extraParam); } - // A splittable DoFn can not have any other extra context parameters. + // The allowed parameters depend on whether this DoFn is splittable if (methodContext.hasRestrictionTrackerParameter()) { - errors.checkArgument( - Iterables.all( - methodContext.getExtraParameters(), - Predicates.or( - Predicates.instanceOf(RestrictionTrackerParameter.class), - Predicates.instanceOf(ProcessContextParameter.class))), - "Splittable %s @%s must have only %s and %s parameters, but has: %s", - DoFn.class.getSimpleName(), - DoFn.ProcessElement.class.getSimpleName(), - DoFn.ProcessContext.class.getSimpleName(), - RestrictionTracker.class.getSimpleName(), - methodContext.getExtraParameters()); + for (Parameter parameter : methodContext.getExtraParameters()) { + checkParameterOneOf(errors, parameter, ALLOWED_SPLITTABLE_PROCESS_ELEMENT_PARAMETERS); + } + } else { + for (Parameter parameter : methodContext.getExtraParameters()) { + checkParameterOneOf(errors, parameter, ALLOWED_NON_SPLITTABLE_PROCESS_ELEMENT_PARAMETERS); + } } return DoFnSignature.ProcessElementMethod.create( @@ -703,6 +740,21 @@ static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( DoFn.ProcessContinuation.class.equals(m.getReturnType())); } + private static void checkParameterOneOf( + ErrorReporter errors, + Parameter parameter, + Collection> allowedParameterClasses) { + + for (Class paramClass : allowedParameterClasses) { + if (paramClass.isAssignableFrom(parameter.getClass())) { + return; + } + } + + // If we get here, none matched + errors.throwIllegalArgument("Illegal parameter type: %s", parameter); + } + private static Parameter analyzeExtraParameter( ErrorReporter methodErrors, FnAnalysisContext fnContext, @@ -714,6 +766,7 @@ private static Parameter analyzeExtraParameter( TypeDescriptor expectedProcessContextT = doFnProcessContextTypeOf(inputT, outputT); TypeDescriptor expectedContextT = doFnContextTypeOf(inputT, outputT); + TypeDescriptor expectedOnTimerContextT = doFnOnTimerContextTypeOf(inputT, outputT); TypeDescriptor expectedInputProviderT = inputProviderTypeOf(inputT); TypeDescriptor expectedOutputReceiverT = outputReceiverTypeOf(outputT); @@ -732,6 +785,11 @@ private static Parameter analyzeExtraParameter( "Must take %s as the Context argument", formatType(expectedContextT)); return Parameter.context(); + } else if (rawType.equals(DoFn.OnTimerContext.class)) { + methodErrors.checkArgument(paramT.equals(expectedOnTimerContextT), + "Must take %s as the OnTimerContext argument", + formatType(expectedOnTimerContextT)); + return Parameter.onTimerContext(); } else if (BoundedWindow.class.isAssignableFrom(rawType)) { methodErrors.checkArgument( !methodContext.hasWindowParameter(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java index 91f2d1b9fca4..7b594c97f53c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java @@ -90,7 +90,8 @@ private void method( @Test public void testSplittableProcessElementMustNotHaveOtherParams() throws Exception { thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("must have only ProcessContext and RestrictionTracker parameters"); + thrown.expectMessage("Illegal parameter"); + thrown.expectMessage("BoundedWindow"); DoFnSignature.ProcessElementMethod signature = analyzeProcessElementMethod( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index 1381cd9d1e33..69d40582c589 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -441,6 +441,53 @@ public void onFoo() {} assertThat(decl.field().getName(), equalTo("bizzle")); } + @Test + public void testSimpleTimerWithContext() throws Exception { + DoFnSignature sig = + DoFnSignatures.getSignature( + new DoFn, Long>() { + @TimerId("foo") + private final TimerSpec bizzle = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void foo(ProcessContext context) {} + + @OnTimer("foo") + public void onFoo(OnTimerContext c) {} + }.getClass()); + + assertThat(sig.timerDeclarations().size(), equalTo(1)); + DoFnSignature.TimerDeclaration decl = sig.timerDeclarations().get("foo"); + + assertThat(decl.id(), equalTo("foo")); + assertThat(decl.field().getName(), equalTo("bizzle")); + + assertThat( + sig.onTimerMethods().get("foo").extraParameters().get(0), + equalTo((Parameter) Parameter.onTimerContext())); + } + + @Test + public void testProcessElementWithOnTimerContextRejected() throws Exception { + thrown.expect(IllegalArgumentException.class); + // The message should at least mention @ProcessElement and OnTimerContext + thrown.expectMessage("@" + DoFn.ProcessElement.class.getSimpleName()); + thrown.expectMessage(DoFn.OnTimerContext.class.getSimpleName()); + + DoFnSignature sig = + DoFnSignatures.getSignature( + new DoFn, Long>() { + @TimerId("foo") + private final TimerSpec bizzle = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void foo(ProcessContext context, OnTimerContext bogus) {} + + @OnTimer("foo") + public void onFoo() {} + }.getClass()); + } + @Test public void testSimpleTimerIdNamedDoFn() throws Exception { class DoFnForTestSimpleTimerIdNamedDoFn extends DoFn, Long> { From 2883062eebe8dba849ab89627f6aeb53266ac1a8 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:10:21 -0800 Subject: [PATCH 111/279] Access to OnTimerContext via DoFnInvokers.ArgumentProvider --- .../apache/beam/runners/core/SimpleDoFnRunner.java | 13 +++++++++++++ .../apache/beam/runners/core/SplittableParDo.java | 5 +++++ .../apache/beam/sdk/transforms/DoFnAdapters.java | 12 ++++++++++++ .../org/apache/beam/sdk/transforms/DoFnTester.java | 7 +++++++ .../beam/sdk/transforms/reflect/DoFnInvoker.java | 8 ++++++++ 5 files changed, 45 insertions(+) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 68751f09a234..0d41a8d0270e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.DoFn.InputProvider; +import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -402,6 +403,12 @@ public ProcessContext processContext(DoFn doFn) { "Cannot access ProcessContext outside of @Processelement method."); } + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Cannot access OnTimerContext outside of @OnTimer methods."); + } + @Override public InputProvider inputProvider() { throw new UnsupportedOperationException("InputProvider is for testing only."); @@ -588,6 +595,12 @@ public DoFn.ProcessContext processContext(DoFn return this; } + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Cannot access OnTimerContext outside of @OnTimer methods."); + } + @Override public InputProvider inputProvider() { throw new UnsupportedOperationException("InputProvider parameters are not supported."); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 78f373bfd5aa..580e842cc66a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -662,6 +662,11 @@ public DoFn.ProcessContext processContext(DoFn doFn) { return processContext; } + @Override + public DoFn.OnTimerContext onTimerContext(DoFn doFn) { + throw new IllegalStateException("Unexpected extra context access on a splittable DoFn"); + } + @Override public DoFn.InputProvider inputProvider() { // DoFnSignatures should have verified that this DoFn doesn't access extra context. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index 6ee42e7a58ef..e15b08b46d79 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -23,6 +23,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn.Context; +import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -344,6 +345,12 @@ public ProcessContext processContext(DoFn doFn) { "Can only get a ProcessContext in processElement"); } + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Timers are not supported for OldDoFn"); + } + @Override public WindowingInternals windowingInternals() { // The OldDoFn doesn't allow us to ask for these outside ProcessElements, so this @@ -459,6 +466,11 @@ public ProcessContext processContext(DoFn doFn) { return this; } + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException("Timers are not supported for OldDoFn"); + } + @Override public WindowingInternals windowingInternals() { return context.windowingInternals(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 43896c599404..93b3f5954898 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ValueInSingleWindow; import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; @@ -315,6 +316,12 @@ public DoFn.ProcessContext processContext(DoFn return processContext; } + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException( + "DoFnTester doesn't support timers yet."); + } + @Override public DoFn.InputProvider inputProvider() { throw new UnsupportedOperationException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index 5e61bddbe4d9..97ac9d3d53c7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -112,6 +112,9 @@ interface ArgumentProvider { /** Provide a {@link DoFn.ProcessContext} to use with the given {@link DoFn}. */ DoFn.ProcessContext processContext(DoFn doFn); + /** Provide a {@link DoFn.OnTimerContext} to use with the given {@link DoFn}. */ + DoFn.OnTimerContext onTimerContext(DoFn doFn); + /** A placeholder for testing purposes. */ InputProvider inputProvider(); @@ -161,6 +164,11 @@ public DoFn.Context context(DoFn doFn) { return null; } + @Override + public DoFn.OnTimerContext onTimerContext(DoFn doFn) { + return null; + } + @Override public InputProvider inputProvider() { return null; From a26ed134bc57970ed83156f93d660a637465a9d6 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:19:32 -0800 Subject: [PATCH 112/279] Support OnTimerContext in ByteBuddyDoFnInvokerFactory --- .../transforms/reflect/ByteBuddyDoFnInvokerFactory.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java index 3480603fdaef..01ddd86b34ba 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java @@ -85,6 +85,7 @@ public class ByteBuddyDoFnInvokerFactory implements DoFnInvokerFactory { public static final String CONTEXT_PARAMETER_METHOD = "context"; public static final String PROCESS_CONTEXT_PARAMETER_METHOD = "processContext"; + public static final String ON_TIMER_CONTEXT_PARAMETER_METHOD = "onTimerContext"; public static final String WINDOW_PARAMETER_METHOD = "window"; public static final String INPUT_PROVIDER_PARAMETER_METHOD = "inputProvider"; public static final String OUTPUT_RECEIVER_PARAMETER_METHOD = "outputReceiver"; @@ -556,7 +557,11 @@ public StackManipulation dispatch(ProcessContextParameter p) { @Override public StackManipulation dispatch(OnTimerContextParameter p) { - throw new UnsupportedOperationException("OnTimerContext is not yet supported."); + return new StackManipulation.Compound( + pushDelegate, + MethodInvocation.invoke( + getExtraContextFactoryMethodDescription( + ON_TIMER_CONTEXT_PARAMETER_METHOD, DoFn.class))); } @Override From 09e2f309c9554f58d2b9a2be5f83ef2751d9d40b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 14:28:39 -0800 Subject: [PATCH 113/279] Move CopyOnAccessStateInternals to runners/direct --- .../CopyOnAccessInMemoryStateInternals.java | 46 ++++++++++++------- .../direct/DirectExecutionContext.java | 1 - .../runners/direct/EvaluationContext.java | 1 - .../GroupAlsoByWindowEvaluatorFactory.java | 1 - .../beam/runners/direct/ParDoEvaluator.java | 1 - .../runners/direct/StepTransformResult.java | 1 - .../beam/runners/direct/TransformResult.java | 1 - ...opyOnAccessInMemoryStateInternalsTest.java | 12 ++++- .../runners/direct/EvaluationContextTest.java | 1 - .../StatefulParDoEvaluatorFactoryTest.java | 1 - .../util/state/InMemoryStateInternals.java | 33 +++++++++---- 11 files changed, 66 insertions(+), 33 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util/state => runners/direct-java/src/main/java/org/apache/beam/runners/direct}/CopyOnAccessInMemoryStateInternals.java (90%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util/state => runners/direct-java/src/test/java/org/apache/beam/runners/direct}/CopyOnAccessInMemoryStateInternalsTest.java (97%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java similarity index 90% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java rename to runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java index 3ca00a910a96..e486a754a4ff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.runners.direct; import static com.google.common.base.Preconditions.checkState; @@ -32,8 +32,24 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; +import org.apache.beam.sdk.util.state.AccumulatorCombiningState; +import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryBag; +import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryCombiningValue; import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryState; +import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryStateBinder; +import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryValue; +import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryWatermarkHold; +import org.apache.beam.sdk.util.state.State; +import org.apache.beam.sdk.util.state.StateContext; +import org.apache.beam.sdk.util.state.StateContexts; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateTable; +import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTag.StateBinder; +import org.apache.beam.sdk.util.state.ValueState; +import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; /** @@ -262,11 +278,11 @@ public WatermarkHoldState bindWatermark( if (containedInUnderlying(namespace, address)) { @SuppressWarnings("unchecked") InMemoryState> existingState = - (InMemoryStateInternals.InMemoryState>) + (InMemoryState>) underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryStateInternals.InMemoryWatermarkHold<>( + return new InMemoryWatermarkHold<>( outputTimeFn); } } @@ -277,11 +293,11 @@ public ValueState bindValue( if (containedInUnderlying(namespace, address)) { @SuppressWarnings("unchecked") InMemoryState> existingState = - (InMemoryStateInternals.InMemoryState>) + (InMemoryState>) underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryStateInternals.InMemoryValue<>(); + return new InMemoryValue<>(); } } @@ -294,12 +310,11 @@ public ValueState bindValue( @SuppressWarnings("unchecked") InMemoryState> existingState = ( - InMemoryStateInternals - .InMemoryState>) underlying.get().get(namespace, address, c); + InMemoryState>) underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryStateInternals.InMemoryCombiningValue<>( + return new InMemoryCombiningValue<>( key, combineFn.asKeyedFn()); } } @@ -310,11 +325,11 @@ public BagState bindBag( if (containedInUnderlying(namespace, address)) { @SuppressWarnings("unchecked") InMemoryState> existingState = - (InMemoryStateInternals.InMemoryState>) + (InMemoryState>) underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryStateInternals.InMemoryBag<>(); + return new InMemoryBag<>(); } } @@ -328,12 +343,11 @@ public BagState bindBag( @SuppressWarnings("unchecked") InMemoryState> existingState = ( - InMemoryStateInternals - .InMemoryState>) underlying.get().get(namespace, address, c); + InMemoryState>) underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryStateInternals.InMemoryCombiningValue<>(key, combineFn); + return new InMemoryCombiningValue<>(key, combineFn); } } @@ -446,7 +460,7 @@ public InMemoryStateBinderFactory(K key) { @Override public StateBinder forNamespace(StateNamespace namespace, StateContext c) { - return new InMemoryStateInternals.InMemoryStateBinder<>(key, c); + return new InMemoryStateBinder<>(key, c); } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 8cec8f7417c8..c6051f08dfb1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.BaseExecutionContext; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; /** * Execution Context for the {@link DirectRunner}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index b5a23d7cbc97..230d91b52dff 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -48,7 +48,6 @@ import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 87cbbcde54f9..bb11923fd1cd 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -50,7 +50,6 @@ import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 504ddc462aab..a915cf0bc990 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java index d58b027dd8a1..01b2a7261f07 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.joda.time.Instant; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java index b4797b01ce15..8bb5f9355860 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.joda.time.Instant; /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java similarity index 97% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java rename to runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java index ad70bcafe753..deefc68489dc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.runners.direct; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; @@ -39,6 +39,16 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; +import org.apache.beam.sdk.util.state.AccumulatorCombiningState; +import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaceForTest; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.apache.beam.sdk.util.state.StateTag; +import org.apache.beam.sdk.util.state.StateTags; +import org.apache.beam.sdk.util.state.ValueState; +import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index a2bb15ecb037..f11c370ac9d2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -58,7 +58,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 06c85ef06914..7c086a1b629e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -51,7 +51,6 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java index efb270cfa9ec..66118374d808 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryStateInternals.java @@ -56,7 +56,11 @@ public K getKey() { return key; } - interface InMemoryState> { + /** + * Interface common to all in-memory state cells. Includes ability to see whether a cell has been + * cleared and the ability to create a clone of the contents. + */ + public interface InMemoryState> { boolean isCleared(); T copy(); } @@ -94,11 +98,11 @@ public T state( /** * A {@link StateBinder} that returns In Memory {@link State} objects. */ - static class InMemoryStateBinder implements StateBinder { + public static class InMemoryStateBinder implements StateBinder { private final K key; private final StateContext c; - InMemoryStateBinder(K key, StateContext c) { + public InMemoryStateBinder(K key, StateContext c) { this.key = key; this.c = c; } @@ -150,7 +154,11 @@ public WatermarkHoldState bindWatermark( } } - static final class InMemoryValue implements ValueState, InMemoryState> { + /** + * An {@link InMemoryState} implementation of {@link ValueState}. + */ + public static final class InMemoryValue + implements ValueState, InMemoryState> { private boolean isCleared = true; private T value = null; @@ -194,7 +202,10 @@ public boolean isCleared() { } } - static final class InMemoryWatermarkHold + /** + * An {@link InMemoryState} implementation of {@link WatermarkHoldState}. + */ + public static final class InMemoryWatermarkHold implements WatermarkHoldState, InMemoryState> { private final OutputTimeFn outputTimeFn; @@ -267,7 +278,10 @@ public InMemoryWatermarkHold copy() { } } - static final class InMemoryCombiningValue + /** + * An {@link InMemoryState} implementation of {@link AccumulatorCombiningState}. + */ + public static final class InMemoryCombiningValue implements AccumulatorCombiningState, InMemoryState> { private final K key; @@ -275,7 +289,7 @@ static final class InMemoryCombiningValue private final KeyedCombineFn combineFn; private AccumT accum; - InMemoryCombiningValue( + public InMemoryCombiningValue( K key, KeyedCombineFn combineFn) { this.key = key; this.combineFn = combineFn; @@ -353,7 +367,10 @@ public InMemoryCombiningValue copy() { } } - static final class InMemoryBag implements BagState, InMemoryState> { + /** + * An {@link InMemoryState} implementation of {@link BagState}. + */ + public static final class InMemoryBag implements BagState, InMemoryState> { private List contents = new ArrayList<>(); @Override From 44e17d1c97babd487584cc78690505bdf57704b2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 14:17:01 -0800 Subject: [PATCH 114/279] Remove misc occurrences of OldDoFn --- .../beam/sdk/AggregatorPipelineExtractor.java | 5 ++-- .../sdk/transforms/AggregatorRetriever.java | 2 +- .../apache/beam/sdk/transforms/Combine.java | 4 +-- .../beam/sdk/util/ExecutionContext.java | 8 +++--- .../sdk/AggregatorPipelineExtractorTest.java | 20 ++++++------- .../beam/sdk/transforms/DoFnTesterTest.java | 2 +- .../sdk/transforms/ParDoLifecycleTest.java | 28 +++++++++---------- 7 files changed, 35 insertions(+), 34 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java index d2130d06f46f..ade5978b4eb0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java @@ -70,9 +70,10 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { private Collection> getAggregators(PTransform transform) { if (transform != null) { if (transform instanceof ParDo.Bound) { - return AggregatorRetriever.getAggregators(((ParDo.Bound) transform).getFn()); + return AggregatorRetriever.getAggregators(((ParDo.Bound) transform).getNewFn()); } else if (transform instanceof ParDo.BoundMulti) { - return AggregatorRetriever.getAggregators(((ParDo.BoundMulti) transform).getFn()); + return AggregatorRetriever.getAggregators( + ((ParDo.BoundMulti) transform).getNewFn()); } } return Collections.emptyList(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java index abed843d9fa7..ce47e22fda06 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java @@ -30,7 +30,7 @@ private AggregatorRetriever() { /** * Returns the {@link Aggregator Aggregators} created by the provided {@link OldDoFn}. */ - public static Collection> getAggregators(OldDoFn fn) { + public static Collection> getAggregators(DoFn fn) { return fn.getAggregators(); } } 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 be063e2f9c84..4127d948b957 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 @@ -2392,8 +2392,8 @@ public PCollection> apply( PCollection>> input) { PCollection> output = input.apply(ParDo.of( - new OldDoFn>, KV>() { - @Override + new DoFn>, KV>() { + @ProcessElement public void processElement(final ProcessContext c) { K key = c.element().getKey(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java index f2a79bd0f7bf..4429d7614eab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java @@ -41,14 +41,14 @@ public interface ExecutionContext { /** * Hook for subclasses to implement that will be called whenever - * {@link org.apache.beam.sdk.transforms.OldDoFn.Context#output} + * {@link org.apache.beam.sdk.transforms.DoFn.Context#output} * is called. */ void noteOutput(WindowedValue output); /** * Hook for subclasses to implement that will be called whenever - * {@link org.apache.beam.sdk.transforms.OldDoFn.Context#sideOutput} + * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput} * is called. */ void noteSideOutput(TupleTag tag, WindowedValue output); @@ -70,14 +70,14 @@ public interface StepContext { /** * Hook for subclasses to implement that will be called whenever - * {@link org.apache.beam.sdk.transforms.OldDoFn.Context#output} + * {@link org.apache.beam.sdk.transforms.DoFn.Context#output} * is called. */ void noteOutput(WindowedValue output); /** * Hook for subclasses to implement that will be called whenever - * {@link org.apache.beam.sdk.transforms.OldDoFn.Context#sideOutput} + * {@link org.apache.beam.sdk.transforms.DoFn.Context#sideOutput} * is called. */ void noteSideOutput(TupleTag tag, WindowedValue output); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java index b4de768b5998..c4e9b8a30519 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java @@ -33,9 +33,9 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; @@ -68,7 +68,7 @@ public void testGetAggregatorStepsWithParDoBoundExtractsSteps() { @SuppressWarnings("rawtypes") ParDo.Bound bound = mock(ParDo.Bound.class, "Bound"); AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); - when(bound.getFn()).thenReturn(fn); + when(bound.getNewFn()).thenReturn(fn); Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinIntegerFn()); @@ -96,7 +96,7 @@ public void testGetAggregatorStepsWithParDoBoundMultiExtractsSteps() { @SuppressWarnings("rawtypes") ParDo.BoundMulti bound = mock(ParDo.BoundMulti.class, "BoundMulti"); AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); - when(bound.getFn()).thenReturn(fn); + when(bound.getNewFn()).thenReturn(fn); Aggregator aggregatorOne = fn.addAggregator(new Max.MaxLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn()); @@ -126,8 +126,8 @@ public void testGetAggregatorStepsWithOneAggregatorInMultipleStepsAddsSteps() { @SuppressWarnings("rawtypes") ParDo.BoundMulti otherBound = mock(ParDo.BoundMulti.class, "otherBound"); AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); - when(bound.getFn()).thenReturn(fn); - when(otherBound.getFn()).thenReturn(fn); + when(bound.getNewFn()).thenReturn(fn); + when(otherBound.getNewFn()).thenReturn(fn); Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn()); @@ -162,7 +162,7 @@ public void testGetAggregatorStepsWithDifferentStepsAddsSteps() { AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); - when(bound.getFn()).thenReturn(fn); + when(bound.getNewFn()).thenReturn(fn); @SuppressWarnings("rawtypes") ParDo.BoundMulti otherBound = mock(ParDo.BoundMulti.class, "otherBound"); @@ -170,7 +170,7 @@ public void testGetAggregatorStepsWithDifferentStepsAddsSteps() { AggregatorProvidingDoFn otherFn = new AggregatorProvidingDoFn<>(); Aggregator aggregatorTwo = otherFn.addAggregator(new Sum.SumDoubleFn()); - when(otherBound.getFn()).thenReturn(otherFn); + when(otherBound.getNewFn()).thenReturn(otherFn); TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class); when(transformNode.getTransform()).thenReturn(bound); @@ -208,7 +208,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { } } - private static class AggregatorProvidingDoFn extends OldDoFn { + private static class AggregatorProvidingDoFn extends DoFn { public Aggregator addAggregator( CombineFn combiner) { return createAggregator(randomName(), combiner); @@ -218,8 +218,8 @@ private String randomName() { return UUID.randomUUID().toString(); } - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception { + @ProcessElement + public void processElement(DoFn.ProcessContext c) throws Exception { fail(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index b47465eab436..2dafa27d99d7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -399,7 +399,7 @@ public void processElement(ProcessContext c) throws Exception { /** * A {@link DoFn} that adds values to an aggregator and converts input to String in - * {@link OldDoFn#processElement}. + * {@link DoFn.ProcessElement @ProcessElement}. */ private static class CounterDoFn extends DoFn { Aggregator agg = createAggregator("ctr", new Sum.SumLongFn()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java index f69c867bbe22..9bc8a6480555 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java @@ -50,7 +50,7 @@ public void testOldFnCallSequence() { PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) - .apply(ParDo.of(new CallSequenceEnforcingOldFn())); + .apply(ParDo.of(new CallSequenceEnforcingDoFn())); p.run(); } @@ -62,19 +62,19 @@ public void testOldFnCallSequenceMulti() { PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) - .apply(ParDo.of(new CallSequenceEnforcingOldFn()) + .apply(ParDo.of(new CallSequenceEnforcingDoFn()) .withOutputTags(new TupleTag() {}, TupleTagList.empty())); p.run(); } - private static class CallSequenceEnforcingOldFn extends OldDoFn { + private static class CallSequenceEnforcingDoFn extends DoFn { private boolean setupCalled = false; private int startBundleCalls = 0; private int finishBundleCalls = 0; private boolean teardownCalled = false; - @Override + @Setup public void setup() { assertThat("setup should not be called twice", setupCalled, is(false)); assertThat("setup should be called before startBundle", startBundleCalls, equalTo(0)); @@ -83,7 +83,7 @@ public void setup() { setupCalled = true; } - @Override + @StartBundle public void startBundle(Context c) { assertThat("setup should have been called", setupCalled, is(true)); assertThat( @@ -94,7 +94,7 @@ public void startBundle(Context c) { startBundleCalls++; } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); assertThat( @@ -104,7 +104,7 @@ public void processElement(ProcessContext c) throws Exception { assertThat("teardown should not have been called", teardownCalled, is(false)); } - @Override + @FinishBundle public void finishBundle(Context c) { assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); assertThat( @@ -115,7 +115,7 @@ public void finishBundle(Context c) { finishBundleCalls++; } - @Override + @Teardown public void teardown() { assertThat(setupCalled, is(true)); assertThat(startBundleCalls, anyOf(equalTo(finishBundleCalls))); @@ -345,7 +345,7 @@ public void testWithContextTeardownCalledAfterExceptionInFinishBundle() { } } - private static class ExceptionThrowingOldFn extends OldDoFn { + private static class ExceptionThrowingOldFn extends DoFn { static AtomicBoolean teardownCalled = new AtomicBoolean(false); private final MethodForException toThrow; @@ -355,22 +355,22 @@ private ExceptionThrowingOldFn(MethodForException toThrow) { this.toThrow = toThrow; } - @Override + @Setup public void setup() throws Exception { throwIfNecessary(MethodForException.SETUP); } - @Override + @StartBundle public void startBundle(Context c) throws Exception { throwIfNecessary(MethodForException.START_BUNDLE); } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { throwIfNecessary(MethodForException.PROCESS_ELEMENT); } - @Override + @FinishBundle public void finishBundle(Context c) throws Exception { throwIfNecessary(MethodForException.FINISH_BUNDLE); } @@ -382,7 +382,7 @@ private void throwIfNecessary(MethodForException method) throws Exception { } } - @Override + @Teardown public void teardown() { if (!thrown) { fail("Excepted to have a processing method throw an exception"); From 4d607b5a594bbf2be76626200d989a9e65ba3da9 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 13:33:04 -0800 Subject: [PATCH 115/279] Rename PTransform.apply to PTransform.expand --- .../org/apache/beam/examples/WordCount.java | 2 +- .../beam/examples/complete/AutoComplete.java | 6 +-- .../apache/beam/examples/complete/TfIdf.java | 6 +-- .../complete/TopWikipediaSessions.java | 6 +-- .../examples/complete/TrafficMaxLaneFlow.java | 4 +- .../beam/examples/complete/TrafficRoutes.java | 4 +- .../examples/cookbook/BigQueryTornadoes.java | 2 +- .../cookbook/CombinePerKeyExamples.java | 2 +- .../examples/cookbook/FilterExamples.java | 2 +- .../examples/cookbook/MaxPerKeyExamples.java | 2 +- .../examples/cookbook/TriggerExample.java | 4 +- .../examples/complete/AutoCompleteTest.java | 2 +- .../examples/complete/game/GameStats.java | 2 +- .../examples/complete/game/LeaderBoard.java | 4 +- .../examples/complete/game/UserScore.java | 2 +- .../complete/game/utils/WriteToBigQuery.java | 2 +- .../game/utils/WriteWindowedToBigQuery.java | 2 +- .../apache/beam/runners/apex/ApexRunner.java | 10 ++-- .../beam/runners/core/AssignWindows.java | 2 +- .../core/GroupByKeyViaGroupByKeyOnly.java | 8 +-- .../beam/runners/core/SplittableParDo.java | 6 +-- .../core/UnboundedReadFromBoundedSource.java | 2 +- .../beam/runners/direct/DirectGroupByKey.java | 6 +-- .../runners/direct/ForwardingPTransform.java | 6 +-- .../direct/ParDoMultiOverrideFactory.java | 4 +- .../ParDoSingleViaMultiOverrideFactory.java | 2 +- .../direct/TestStreamEvaluatorFactory.java | 2 +- .../runners/direct/ViewEvaluatorFactory.java | 4 +- .../direct/WriteWithShardingFactory.java | 4 +- .../runners/direct/CommittedResultTest.java | 2 +- .../direct/DirectGraphVisitorTest.java | 2 +- .../direct/ForwardingPTransformTest.java | 4 +- .../KeyedPValueTrackingVisitorTest.java | 4 +- .../beam/runners/flink/examples/TFIDF.java | 6 +-- .../runners/flink/examples/WordCount.java | 2 +- .../examples/streaming/AutoComplete.java | 6 +-- .../beam/runners/flink/FlinkRunner.java | 14 ++--- .../beam/runners/dataflow/DataflowRunner.java | 51 ++++++++++--------- .../dataflow/internal/AssignWindows.java | 2 +- ...ataflowUnboundedReadFromBoundedSource.java | 2 +- .../DataflowPipelineTranslatorTest.java | 8 +-- .../runners/dataflow/DataflowRunnerTest.java | 2 +- .../transforms/DataflowGroupByKeyTest.java | 2 +- .../dataflow/transforms/DataflowViewTest.java | 2 +- .../runners/spark/examples/WordCount.java | 2 +- .../beam/runners/spark/io/ConsoleIO.java | 2 +- .../beam/runners/spark/io/CreateStream.java | 2 +- .../runners/spark/io/hadoop/HadoopIO.java | 4 +- .../translation/StorageLevelPTransform.java | 2 +- .../util/SinglePrimitiveOutputPTransform.java | 2 +- .../java/org/apache/beam/sdk/io/AvroIO.java | 4 +- .../io/BoundedReadFromUnboundedSource.java | 2 +- .../org/apache/beam/sdk/io/CountingInput.java | 4 +- .../java/org/apache/beam/sdk/io/PubsubIO.java | 4 +- .../beam/sdk/io/PubsubUnboundedSink.java | 2 +- .../beam/sdk/io/PubsubUnboundedSource.java | 2 +- .../java/org/apache/beam/sdk/io/Read.java | 4 +- .../java/org/apache/beam/sdk/io/TextIO.java | 4 +- .../java/org/apache/beam/sdk/io/Write.java | 2 +- .../beam/sdk/runners/PipelineRunner.java | 2 +- .../beam/sdk/testing/GatherAllPanes.java | 2 +- .../org/apache/beam/sdk/testing/PAssert.java | 12 ++--- .../apache/beam/sdk/testing/TestStream.java | 2 +- .../sdk/transforms/ApproximateUnique.java | 4 +- .../apache/beam/sdk/transforms/Combine.java | 10 ++-- .../org/apache/beam/sdk/transforms/Count.java | 2 +- .../apache/beam/sdk/transforms/Create.java | 4 +- .../apache/beam/sdk/transforms/Distinct.java | 4 +- .../apache/beam/sdk/transforms/Filter.java | 2 +- .../beam/sdk/transforms/FlatMapElements.java | 2 +- .../apache/beam/sdk/transforms/Flatten.java | 4 +- .../beam/sdk/transforms/GroupByKey.java | 2 +- .../org/apache/beam/sdk/transforms/Keys.java | 2 +- .../apache/beam/sdk/transforms/KvSwap.java | 2 +- .../apache/beam/sdk/transforms/Latest.java | 4 +- .../beam/sdk/transforms/MapElements.java | 2 +- .../beam/sdk/transforms/PTransform.java | 4 +- .../org/apache/beam/sdk/transforms/ParDo.java | 4 +- .../apache/beam/sdk/transforms/Partition.java | 2 +- .../org/apache/beam/sdk/transforms/Regex.java | 14 ++--- .../apache/beam/sdk/transforms/Sample.java | 2 +- .../apache/beam/sdk/transforms/Values.java | 2 +- .../org/apache/beam/sdk/transforms/View.java | 12 ++--- .../apache/beam/sdk/transforms/WithKeys.java | 2 +- .../beam/sdk/transforms/WithTimestamps.java | 2 +- .../sdk/transforms/join/CoGroupByKey.java | 2 +- .../beam/sdk/transforms/windowing/Window.java | 4 +- .../org/apache/beam/sdk/util/Reshuffle.java | 2 +- .../org/apache/beam/sdk/PipelineTest.java | 6 +-- .../beam/sdk/coders/CoderRegistryTest.java | 4 +- .../org/apache/beam/sdk/io/WriteTest.java | 2 +- .../sdk/runners/TransformHierarchyTest.java | 4 +- .../beam/sdk/runners/TransformTreeTest.java | 4 +- .../sdk/transforms/FlatMapElementsTest.java | 2 +- .../beam/sdk/transforms/GroupByKeyTest.java | 2 +- .../beam/sdk/transforms/MapElementsTest.java | 2 +- .../beam/sdk/transforms/PTransformTest.java | 2 +- .../apache/beam/sdk/transforms/ParDoTest.java | 4 +- .../apache/beam/sdk/transforms/ViewTest.java | 2 +- .../display/DisplayDataEvaluatorTest.java | 2 +- .../display/DisplayDataMatchersTest.java | 6 +-- .../transforms/display/DisplayDataTest.java | 2 +- .../transforms/windowing/WindowingTest.java | 2 +- .../apache/beam/sdk/util/StringUtilsTest.java | 6 +-- .../org/apache/beam/sdk/values/PDoneTest.java | 4 +- .../sdk/extensions/sorter/SortValues.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 8 +-- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 4 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 4 +- .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 4 +- .../org/apache/beam/sdk/io/jms/JmsIO.java | 4 +- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 10 ++-- .../beam/sdk/io/mongodb/MongoDbGridFSIO.java | 4 +- .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 4 +- 114 files changed, 238 insertions(+), 237 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 5be0ddc7b54a..d4da54229c6b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -126,7 +126,7 @@ public String apply(KV input) { public static class CountWords extends PTransform, PCollection>> { @Override - public PCollection> apply(PCollection lines) { + public PCollection> expand(PCollection lines) { // Convert lines of text into individual words. PCollection words = lines.apply( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index c556e3fc5fb9..31b06c9f0bc0 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -113,7 +113,7 @@ public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursi } @Override - public PCollection>> apply(PCollection input) { + public PCollection>> expand(PCollection input) { PCollection candidates = input // First count how often each token appears. .apply(new Count.PerElement()) @@ -154,7 +154,7 @@ public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) { } @Override - public PCollection>> apply( + public PCollection>> expand( PCollection input) { return input // For each completion candidate, map it to all prefixes. @@ -209,7 +209,7 @@ public void processElement(ProcessContext c) { } @Override - public PCollectionList>> apply( + public PCollectionList>> expand( PCollection input) { if (minPrefix > 10) { // Base case, partitioning to return the output in the expected format. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java index edf48e7e5419..ea015ae73447 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java @@ -159,7 +159,7 @@ public Coder getDefaultOutputCoder() { } @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { Pipeline pipeline = input.getPipeline(); // Create one TextIO.Read transform for each document @@ -200,7 +200,7 @@ public static class ComputeTfIdf public ComputeTfIdf() { } @Override - public PCollection>> apply( + public PCollection>> expand( PCollection> uriToContent) { // Compute the total number of documents, and @@ -390,7 +390,7 @@ public WriteTfIdf(String output) { } @Override - public PDone apply(PCollection>> wordToUriAndTfIdf) { + public PDone expand(PCollection>> wordToUriAndTfIdf) { return wordToUriAndTfIdf .apply("Format", ParDo.of(new DoFn>, String>() { @ProcessElement diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index d57cc3ab2b98..df7f81e23e65 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -99,7 +99,7 @@ public void processElement(ProcessContext c) { static class ComputeSessions extends PTransform, PCollection>> { @Override - public PCollection> apply(PCollection actions) { + public PCollection> expand(PCollection actions) { return actions .apply(Window.into(Sessions.withGapDuration(Duration.standardHours(1)))) @@ -113,7 +113,7 @@ public PCollection> apply(PCollection actions) { private static class TopPerMonth extends PTransform>, PCollection>>> { @Override - public PCollection>> apply(PCollection> sessions) { + public PCollection>> expand(PCollection> sessions) { return sessions .apply(Window.>into(CalendarWindows.months(1))) @@ -154,7 +154,7 @@ public ComputeTopSessions(double samplingThreshold) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input .apply(ParDo.of(new ExtractUserAndTimestamp())) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index 0c367d484a79..c1032b972089 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -267,7 +267,7 @@ static TableSchema getSchema() { static class MaxLaneFlow extends PTransform>, PCollection> { @Override - public PCollection apply(PCollection> flowInfo) { + public PCollection expand(PCollection> flowInfo) { // stationId, LaneInfo => stationId + max lane flow info PCollection> flowMaxes = flowInfo.apply(Combine.perKey( @@ -289,7 +289,7 @@ public ReadFileAndExtractTimestamps(String inputFile) { } @Override - public PCollection apply(PBegin begin) { + public PCollection expand(PBegin begin) { return begin .apply(TextIO.Read.from(inputFile)) .apply(ParDo.of(new ExtractTimestamps())); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index 14cee4dd2946..9b5d5779bebc 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -274,7 +274,7 @@ static TableSchema getSchema() { static class TrackSpeed extends PTransform>, PCollection> { @Override - public PCollection apply(PCollection> stationSpeed) { + public PCollection expand(PCollection> stationSpeed) { // Apply a GroupByKey transform to collect a list of all station // readings for a given route. PCollection>> timeGroup = stationSpeed.apply( @@ -299,7 +299,7 @@ public ReadFileAndExtractTimestamps(String inputFile) { } @Override - public PCollection apply(PBegin begin) { + public PCollection expand(PBegin begin) { return begin .apply(TextIO.Read.from(inputFile)) .apply(ParDo.of(new ExtractTimestamps())); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java index a4c1a6bb0835..14d0f582b223 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java @@ -108,7 +108,7 @@ public void processElement(ProcessContext c) { static class CountTornadoes extends PTransform, PCollection> { @Override - public PCollection apply(PCollection rows) { + public PCollection expand(PCollection rows) { // row... => month... PCollection tornadoes = rows.apply( diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java index 93eee15eb349..29655ea86127 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java @@ -125,7 +125,7 @@ public void processElement(ProcessContext c) { static class PlaysForWord extends PTransform, PCollection> { @Override - public PCollection apply(PCollection rows) { + public PCollection expand(PCollection rows) { // row... => ... PCollection> words = rows.apply( diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java index 6e6452c13882..fb6b50710ca2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java @@ -153,7 +153,7 @@ public BelowGlobalMean(Integer monthFilter) { @Override - public PCollection apply(PCollection rows) { + public PCollection expand(PCollection rows) { // Extract the mean_temp from each row. PCollection meanTemps = rows.apply( diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java index abc10f310ba2..eabc42b9fbcb 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java @@ -100,7 +100,7 @@ public void processElement(ProcessContext c) { static class MaxMeanTemp extends PTransform, PCollection> { @Override - public PCollection apply(PCollection rows) { + public PCollection expand(PCollection rows) { // row... => ... PCollection> temps = rows.apply( diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java index d965d4a0214d..bf3afca51485 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java @@ -171,7 +171,7 @@ static class CalculateTotalFlow } @Override - public PCollectionList apply(PCollection> flowInfo) { + public PCollectionList expand(PCollection> flowInfo) { // Concept #1: The default triggering behavior // By default Beam uses a trigger which fires when the watermark has passed the end of the @@ -332,7 +332,7 @@ public TotalFlow(String triggerType) { } @Override - public PCollection apply(PCollection> flowInfo) { + public PCollection expand(PCollection> flowInfo) { PCollection>> flowPerFreeway = flowInfo .apply(GroupByKey.create()); diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java index 5dbfa7046ed8..d7d4dc6c37d9 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java @@ -168,7 +168,7 @@ private static List parseList(String... entries) { private static class ReifyTimestamps extends PTransform>, PCollection> { @Override - public PCollection apply(PCollection> input) { + public PCollection expand(PCollection> input) { return input.apply(ParDo.of(new DoFn, T>() { @ProcessElement public void processElement(ProcessContext c) { diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java index f35225217d60..6ad6a236d6aa 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java @@ -110,7 +110,7 @@ public static class CalculateSpammyUsers private static final double SCORE_WEIGHT = 2.5; @Override - public PCollection> apply(PCollection> userScores) { + public PCollection> expand(PCollection> userScores) { // Get the sum of scores for each user. PCollection> sumScores = userScores diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java index 3275fa0521c3..519bd5f53475 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java @@ -234,7 +234,7 @@ static class CalculateTeamScores } @Override - public PCollection> apply(PCollection infos) { + public PCollection> expand(PCollection infos) { return infos.apply("LeaderboardTeamFixedWindows", Window.into(FixedWindows.of(teamWindowDuration)) // We will get early (speculative) results as well as cumulative @@ -267,7 +267,7 @@ static class CalculateUserScores } @Override - public PCollection> apply(PCollection input) { + public PCollection> expand(PCollection input) { return input.apply("LeaderboardUserGlobalWindow", Window.into(new GlobalWindows()) // Get periodic results every ten minutes. diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java index 2bca7fcab798..cb81a7e65428 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java @@ -160,7 +160,7 @@ public static class ExtractAndSumScore } @Override - public PCollection> apply( + public PCollection> expand( PCollection gameInfo) { return gameInfo diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java index 89fc2712fb34..1f33915b7a30 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java @@ -118,7 +118,7 @@ protected TableSchema getSchema() { } @Override - public PDone apply(PCollection teamAndScore) { + public PDone expand(PCollection teamAndScore) { return teamAndScore .apply("ConvertToRow", ParDo.of(new BuildRowFn())) .apply(BigQueryIO.Write diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java index 4f2e7191f295..c32289f46381 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java @@ -60,7 +60,7 @@ public void processElement(ProcessContext c, BoundedWindow window) { } @Override - public PDone apply(PCollection teamAndScore) { + public PDone expand(PCollection teamAndScore) { return teamAndScore .apply("ConvertToRow", ParDo.of(new BuildRowFn())) .apply(BigQueryIO.Write diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java index 5ce4feff9c80..9507fb985a17 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -165,7 +165,7 @@ public AssignWindowsAndSetStrategy(Window.Bound wrapped) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { WindowingStrategy outputStrategy = wrapped.getOutputStrategyInternal(input.getWindowingStrategy()); @@ -226,7 +226,7 @@ public PCollectionView getView() { } @Override - public PCollectionView apply(PCollection> input) { + public PCollectionView expand(PCollection> input) { return view; } } @@ -252,7 +252,7 @@ public StreamingCombineGloballyAsSingletonView(ApexRunner runner, } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { PCollection combined = input .apply(Combine.globally(transform.getCombineFn()) .withoutDefaults().withFanout(transform.getFanout())); @@ -282,7 +282,7 @@ public StreamingViewAsSingleton(ApexRunner runner, View.AsSingleton transform } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { Combine.Globally combine = Combine .globally(new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); if (!transform.hasDefaultValue()) { @@ -335,7 +335,7 @@ public StreamingViewAsIterable(ApexRunner runner, View.AsIterable transform) } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.iterableView(input.getPipeline(), input.getWindowingStrategy(), input.getCoder()); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java index f2387f57ba39..375932aaf737 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java @@ -40,7 +40,7 @@ public AssignWindows(WindowFn fn) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn<>(fn))); } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index 43047ca0a81f..694c5ebe056e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java @@ -77,7 +77,7 @@ public GroupByKeyViaGroupByKeyOnly(GroupByKey originalTransform) { } @Override - public PCollection>> apply(PCollection> input) { + public PCollection>> expand(PCollection> input) { WindowingStrategy windowingStrategy = input.getWindowingStrategy(); return input @@ -109,7 +109,7 @@ public static class GroupByKeyOnly @SuppressWarnings({"rawtypes", "unchecked"}) @Override - public PCollection>>> apply(PCollection> input) { + public PCollection>>> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); } @@ -128,7 +128,7 @@ private static class SortValuesByTimestamp PCollection>>>, PCollection>>>> { @Override - public PCollection>>> apply( + public PCollection>>> expand( PCollection>>> input) { return input .apply( @@ -225,7 +225,7 @@ public Coder getValueCoder(Coder>>> inputCode } @Override - public PCollection>> apply( + public PCollection>> expand( PCollection>>> input) { @SuppressWarnings("unchecked") KvCoder>> inputKvCoder = diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 580e842cc66a..0bf882bd13a9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -110,7 +110,7 @@ public SplittableParDo(ParDo.BoundMulti parDo) { } @Override - public PCollectionTuple apply(PCollection input) { + public PCollectionTuple expand(PCollection input) { return applyTyped(input); } @@ -179,7 +179,7 @@ private PCollectionTuple applyTyped(PCollection input) { public static class GBKIntoKeyedWorkItems extends PTransform>, PCollection>> { @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded()); } @@ -247,7 +247,7 @@ public TupleTagList getSideOutputTags() { } @Override - public PCollectionTuple apply( + public PCollectionTuple expand( PCollection>> input) { DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java index 29dc57e1f4af..f3f93e1bccec 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java @@ -88,7 +88,7 @@ public UnboundedReadFromBoundedSource(BoundedSource source) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { return input.getPipeline().apply( Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java index 21776e79880d..405d91347514 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java @@ -45,7 +45,7 @@ public PTransform>, PCollection>>> delega } @Override - public PCollection>> apply(PCollection> input) { + public PCollection>> expand(PCollection> input) { @SuppressWarnings("unchecked") KvCoder inputCoder = (KvCoder) input.getCoder(); @@ -79,7 +79,7 @@ public PCollection>> apply(PCollection> input) { static final class DirectGroupByKeyOnly extends PTransform>, PCollection>> { @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded()); } @@ -126,7 +126,7 @@ public Coder getValueCoder(Coder> inputCoder) { } @Override - public PCollection>> apply(PCollection> input) { + public PCollection>> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), outputWindowingStrategy, input.isBounded()); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java index 77311c2bffee..97c09838a1a5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ForwardingPTransform.java @@ -28,15 +28,15 @@ /** * A base class for implementing {@link PTransform} overrides, which behave identically to the * delegate transform but with overridden methods. Implementors are required to implement - * {@link #delegate()}, which returns the object to forward calls to, and {@link #apply(PInput)}. + * {@link #delegate()}, which returns the object to forward calls to, and {@link #expand(PInput)}. */ public abstract class ForwardingPTransform extends PTransform { protected abstract PTransform delegate(); @Override - public OutputT apply(InputT input) { - return delegate().apply(input); + public OutputT expand(InputT input) { + return delegate().expand(input); } @Override diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 9c9256dbff72..8c96e9b59d4f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -73,7 +73,7 @@ public GbkThenStatefulParDo(ParDo.BoundMulti, OutputT> underlyingP } @Override - public PCollectionTuple apply(PCollection> input) { + public PCollectionTuple expand(PCollection> input) { PCollectionTuple outputs = input .apply("Group by key", GroupByKey.create()) @@ -106,7 +106,7 @@ public Coder getDefaultOutputCoder( return underlyingParDo.getDefaultOutputCoder(originalInput, output); } - public PCollectionTuple apply(PCollection>> input) { + public PCollectionTuple expand(PCollection>> input) { PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java index 7f2de667bb60..10530bbc0285 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java @@ -49,7 +49,7 @@ public ParDoSingleViaMulti(ParDo.Bound underlyingParDo) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { // Output tags for ParDo need only be unique up to applied transform TupleTag mainOutputTag = new TupleTag(MAIN_OUTPUT_TAG); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 9df7cdc20c32..3601dbc3d3eb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -170,7 +170,7 @@ private DirectTestStream(TestStream transform) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { PipelineRunner runner = input.getPipeline().getRunner(); checkState( runner instanceof DirectRunner, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index b92ade14655b..460b1c2b694a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -115,7 +115,7 @@ private DirectCreatePCollectionView(CreatePCollectionView og) { } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { return input.apply(WithKeys.of((Void) null)) .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())) .apply(GroupByKey.create()) @@ -145,7 +145,7 @@ public static final class WriteView } @Override - public PCollectionView apply(PCollection> input) { + public PCollectionView expand(PCollection> input) { return og.getView(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index cf535cfe4236..3c8833764954 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -66,7 +66,7 @@ private DynamicallyReshardedWrite(Bound original) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { checkArgument(IsBounded.BOUNDED == input.isBounded(), "%s can only be applied to a Bounded PCollection", getClass().getSimpleName()); @@ -92,7 +92,7 @@ public PDone apply(PCollection input) { // without adding a new Write Transform Node, which would be overwritten the same way, leading // to an infinite recursion. We cannot modify the number of shards, because that is determined // at runtime. - return original.apply(resharded); + return original.expand(resharded); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index 00dca207a097..c6986c0e6774 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -52,7 +52,7 @@ public class CommittedResultTest implements Serializable { private transient AppliedPTransform transform = AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform() { @Override - public PDone apply(PBegin begin) { + public PDone expand(PBegin begin) { throw new IllegalArgumentException("Should never be applied"); } }); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java index 5ad278b11a9f..b88c9a4775f7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java @@ -213,7 +213,7 @@ public void processElement(DoFn.ProcessContext c) transformed.apply( new PTransform() { @Override - public PDone apply(PInput input) { + public PDone expand(PInput input) { return PDone.in(input.getPipeline()); } }); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java index c75adaa343db..6860a58f9993 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ForwardingPTransformTest.java @@ -66,8 +66,8 @@ public void applyDelegates() { PCollection collection = mock(PCollection.class); @SuppressWarnings("unchecked") PCollection output = mock(PCollection.class); - when(delegate.apply(collection)).thenReturn(output); - PCollection result = forwarding.apply(collection); + when(delegate.expand(collection)).thenReturn(output); + PCollection result = forwarding.expand(collection); assertThat(result, equalTo(output)); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index cf65936f77f6..0852cd3f544d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -163,7 +163,7 @@ public void getKeyedPValuesBeforeTraverseThrows() { private static class PrimitiveKeyer extends PTransform, PCollection> { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) .setCoder(input.getCoder()); @@ -172,7 +172,7 @@ public PCollection apply(PCollection input) { private static class CompositeKeyer extends PTransform, PCollection> { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply(new PrimitiveKeyer()).apply(ParDo.of(new IdentityFn())); } } diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java index b946d98c7f8f..89e261b52b30 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java @@ -176,7 +176,7 @@ public Coder getDefaultOutputCoder() { } @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { Pipeline pipeline = input.getPipeline(); // Create one TextIO.Read transform for each document @@ -219,7 +219,7 @@ public static class ComputeTfIdf public ComputeTfIdf() { } @Override - public PCollection>> apply( + public PCollection>> expand( PCollection> uriToContent) { // Compute the total number of documents, and @@ -419,7 +419,7 @@ public WriteTfIdf(String output) { } @Override - public PDone apply(PCollection>> wordToUriAndTfIdf) { + public PDone expand(PCollection>> wordToUriAndTfIdf) { return wordToUriAndTfIdf .apply("Format", ParDo.of(new DoFn>, String>() { private static final long serialVersionUID = 0; diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java index c816442c8e76..b6b3c1a02ebe 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java @@ -72,7 +72,7 @@ public void processElement(ProcessContext c) { public static class CountWords extends PTransform, PCollection>> { @Override - public PCollection> apply(PCollection lines) { + public PCollection> expand(PCollection lines) { // Convert lines of text into individual words. PCollection words = lines.apply( diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java index 1e0c3acdc48d..3405981d6716 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java @@ -82,7 +82,7 @@ public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursi } @Override - public PCollection>> apply(PCollection input) { + public PCollection>> expand(PCollection input) { PCollection candidates = input // First count how often each token appears. .apply(new Count.PerElement()) @@ -129,7 +129,7 @@ public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) { } @Override - public PCollection>> apply( + public PCollection>> expand( PCollection input) { return input // For each completion candidate, map it to all prefixes. @@ -192,7 +192,7 @@ public void processElement(ProcessContext c) { } @Override - public PCollectionList>> apply( + public PCollectionList>> expand( PCollection input) { if (minPrefix > 10) { // Base case, partitioning to return the output in the expected format. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index 0b927347bfe9..7c1284b33bac 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -307,7 +307,7 @@ public StreamingViewAsMap(FlinkRunner runner, View.AsMap transform) { } @Override - public PCollectionView> apply(PCollection> input) { + public PCollectionView> expand(PCollection> input) { PCollectionView> view = PCollectionViews.mapView( input.getPipeline(), @@ -352,7 +352,7 @@ public StreamingViewAsMultimap(FlinkRunner runner, View.AsMultimap transfo } @Override - public PCollectionView>> apply(PCollection> input) { + public PCollectionView>> expand(PCollection> input) { PCollectionView>> view = PCollectionViews.multimapView( input.getPipeline(), @@ -392,7 +392,7 @@ private static class StreamingViewAsList public StreamingViewAsList(FlinkRunner runner, View.AsList transform) {} @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.listView( input.getPipeline(), @@ -423,7 +423,7 @@ private static class StreamingViewAsIterable public StreamingViewAsIterable(FlinkRunner runner, View.AsIterable transform) { } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.iterableView( input.getPipeline(), @@ -465,7 +465,7 @@ public StreamingViewAsSingleton(FlinkRunner runner, View.AsSingleton transfor } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { Combine.Globally combine = Combine.globally( new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); if (!transform.hasDefaultValue()) { @@ -523,7 +523,7 @@ public StreamingCombineGloballyAsSingletonView( } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { PCollection combined = input.apply(Combine.globally(transform.getCombineFn()) .withoutDefaults() @@ -620,7 +620,7 @@ public PCollectionView getView() { } @Override - public PCollectionView apply(PCollection> input) { + public PCollectionView expand(PCollection> input) { return view; } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 40d894800710..00c94d0b0a80 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -756,7 +756,7 @@ private GroupByKeyAndSortValuesOnly() { } @Override - public PCollection>>> apply(PCollection>> input) { + public PCollection>>> expand(PCollection>> input) { PCollection>>> rval = PCollection.>>>createPrimitiveOutputInternal( input.getPipeline(), @@ -814,7 +814,8 @@ private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder ismCoderForHa } @Override - public PCollection>>>> apply(PCollection input) { + public PCollection>>>> expand( + PCollection input) { @SuppressWarnings("unchecked") Coder windowCoder = (Coder) input.getWindowingStrategy().getWindowFn().windowCoder(); @@ -902,7 +903,7 @@ public BatchViewAsSingleton(DataflowRunner runner, View.AsSingleton transform } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { @SuppressWarnings("unchecked") Coder windowCoder = (Coder) input.getWindowingStrategy().getWindowFn().windowCoder(); @@ -993,7 +994,7 @@ public BatchViewAsIterable(DataflowRunner runner, View.AsIterable transform) } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.iterableView( input.getPipeline(), input.getWindowingStrategy(), input.getCoder()); return BatchViewAsList.applyForIterableLike(runner, input, view); @@ -1097,7 +1098,7 @@ public BatchViewAsList(DataflowRunner runner, View.AsList transform) { } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.listView( input.getPipeline(), input.getWindowingStrategy(), input.getCoder()); return applyForIterableLike(runner, input, view); @@ -1265,7 +1266,7 @@ public BatchViewAsMap(DataflowRunner runner, View.AsMap transform) { } @Override - public PCollectionView> apply(PCollection> input) { + public PCollectionView> expand(PCollection> input) { return this.applyInternal(input); } @@ -1406,7 +1407,7 @@ public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder coder) { @Override public PCollection, WindowedValue>>>> - apply(PCollection> input) { + expand(PCollection> input) { @SuppressWarnings("unchecked") Coder windowCoder = (Coder) @@ -1754,7 +1755,7 @@ public BatchViewAsMultimap(DataflowRunner runner, View.AsMultimap transfor } @Override - public PCollectionView>> apply(PCollection> input) { + public PCollectionView>> expand(PCollection> input) { return this.applyInternal(input); } @@ -2056,13 +2057,13 @@ public BatchWrite(DataflowRunner runner, Write.Bound transform) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { if (transform.getSink() instanceof FileBasedSink) { FileBasedSink sink = (FileBasedSink) transform.getSink(); PathValidator validator = runner.options.getPathValidator(); validator.validateOutputFilePrefixSupported(sink.getBaseOutputFilename()); } - return transform.apply(input); + return transform.expand(input); } } @@ -2071,7 +2072,7 @@ public PDone apply(PCollection input) { // ================================================================================ /** - * Suppress application of {@link PubsubUnboundedSource#apply} in streaming mode so that we + * Suppress application of {@link PubsubUnboundedSource#expand} in streaming mode so that we * can instead defer to Windmill's implementation. */ private static class StreamingPubsubIORead extends PTransform> { @@ -2090,7 +2091,7 @@ PubsubUnboundedSource getOverriddenTransform() { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) .setCoder(transform.getElementCoder()); @@ -2155,7 +2156,7 @@ public void translate( } /** - * Suppress application of {@link PubsubUnboundedSink#apply} in streaming mode so that we + * Suppress application of {@link PubsubUnboundedSink#expand} in streaming mode so that we * can instead defer to Windmill's implementation. */ private static class StreamingPubsubIOWrite extends PTransform, PDone> { @@ -2174,7 +2175,7 @@ PubsubUnboundedSink getOverriddenTransform() { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { return PDone.in(input.getPipeline()); } @@ -2252,7 +2253,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection expand(PInput input) { source.validate(); if (source.requiresDeduping()) { @@ -2277,7 +2278,7 @@ private ReadWithIds(UnboundedSource source) { } @Override - public final PCollection> apply(PInput input) { + public final PCollection> expand(PInput input) { return PCollection.>createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED); } @@ -2327,7 +2328,7 @@ private static class Deduplicate // more per-key overhead. private static final int NUM_RESHARD_KEYS = 10000; @Override - public PCollection apply(PCollection> input) { + public PCollection expand(PCollection> input) { return input .apply(WithKeys.of(new SerializableFunction, Integer>() { @Override @@ -2367,7 +2368,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PBegin input) { + public final PCollection expand(PBegin input) { source.validate(); return Pipeline.applyTransform(input, new DataflowUnboundedReadFromBoundedSource<>(source)) @@ -2425,7 +2426,7 @@ public StreamingViewAsMap(DataflowRunner runner, View.AsMap transform) { } @Override - public PCollectionView> apply(PCollection> input) { + public PCollectionView> expand(PCollection> input) { PCollectionView> view = PCollectionViews.mapView( input.getPipeline(), @@ -2470,7 +2471,7 @@ public StreamingViewAsMultimap(DataflowRunner runner, View.AsMultimap tran } @Override - public PCollectionView>> apply(PCollection> input) { + public PCollectionView>> expand(PCollection> input) { PCollectionView>> view = PCollectionViews.multimapView( input.getPipeline(), @@ -2511,7 +2512,7 @@ private static class StreamingViewAsList public StreamingViewAsList(DataflowRunner runner, View.AsList transform) {} @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.listView( input.getPipeline(), @@ -2543,7 +2544,7 @@ private static class StreamingViewAsIterable public StreamingViewAsIterable(DataflowRunner runner, View.AsIterable transform) { } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.iterableView( input.getPipeline(), @@ -2586,7 +2587,7 @@ public StreamingViewAsSingleton(DataflowRunner runner, View.AsSingleton trans } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { Combine.Globally combine = Combine.globally( new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); if (!transform.hasDefaultValue()) { @@ -2644,7 +2645,7 @@ public StreamingCombineGloballyAsSingletonView( } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { PCollection combined = input.apply(Combine.globally(transform.getCombineFn()) .withoutDefaults() @@ -2770,7 +2771,7 @@ public UnsupportedIO(DataflowRunner runner, PubsubUnboundedSink transform) { @Override - public OutputT apply(InputT input) { + public OutputT expand(InputT input) { String mode = input.getPipeline().getOptions().as(StreamingOptions.class).isStreaming() ? "streaming" : "batch"; String name = diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java index 62d4afff960d..68ee7bc7e23c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java @@ -53,7 +53,7 @@ public AssignWindows(Window.Bound transform) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { WindowingStrategy outputStrategy = transform.getOutputStrategyInternal(input.getWindowingStrategy()); if (transform.getWindowFn() != null) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java index 96a35bc188dc..e1eedd8ae774 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java @@ -93,7 +93,7 @@ public DataflowUnboundedReadFromBoundedSource(BoundedSource source) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { return input.getPipeline().apply( Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 95c7132c88d6..ac4f2dff42e9 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -551,7 +551,7 @@ public EmbeddedTransform(Step step) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), @@ -585,7 +585,7 @@ private static class UnrelatedOutputCreator extends PTransform, PCollection> { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { // Apply an operation so that this is a composite transform. input.apply(Count.perElement()); @@ -606,7 +606,7 @@ private static class UnboundOutputCreator extends PTransform, PDone> { @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { // Apply an operation so that this is a composite transform. input.apply(Count.perElement()); @@ -631,7 +631,7 @@ private static class PartiallyBoundOutputCreator public final TupleTag doneTag = new TupleTag<>("done"); @Override - public PCollectionTuple apply(PCollection input) { + public PCollectionTuple expand(PCollection input) { PCollection sum = input.apply(Sum.integersGlobally()); // Fails here when attempting to construct a tuple with an unbound object. diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 5375c95001b5..1959be55ca97 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -840,7 +840,7 @@ public static class TestTransform public boolean translated = false; @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java index 67408ae43574..c9c7806084b9 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java @@ -104,7 +104,7 @@ public void testGroupByKeyServiceUnbounded() { p.apply( new PTransform>>() { @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { return PCollection.>createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java index b9220aff20eb..455868367ff7 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java @@ -93,7 +93,7 @@ private void testViewUnbounded( .apply( new PTransform>>() { @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { return PCollection.>createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java index d7e520724608..b2672b5ea267 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/examples/WordCount.java @@ -85,7 +85,7 @@ public String apply(KV input) { public static class CountWords extends PTransform, PCollection>> { @Override - public PCollection> apply(PCollection lines) { + public PCollection> expand(PCollection lines) { // Convert lines of text into individual words. PCollection words = lines.apply( diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java index b1c567c757d7..0a566331a45f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/ConsoleIO.java @@ -62,7 +62,7 @@ public int getNum() { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { return PDone.in(input.getPipeline()); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java index a08c54ed133e..7ebba90b518f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java @@ -63,7 +63,7 @@ public Iterable> getQueuedValues() { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { // Spark streaming micro batches are bounded by default return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java index 042c316c1919..f2457ced1e08 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java @@ -94,7 +94,7 @@ public Class getKeyClass() { } @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); } @@ -197,7 +197,7 @@ public Map getConfigurationProperties() { } @Override - public PDone apply(PCollection> input) { + public PDone expand(PCollection> input) { checkNotNull( filenamePrefix, "need to set the filename prefix of an HadoopIO.Write transform"); checkNotNull(formatClass, "need to set the format class of an HadoopIO.Write transform"); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java index 6944dbf0a634..30b51e66e4df 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java @@ -29,7 +29,7 @@ public final class StorageLevelPTransform extends PTransform, PCollection> { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java index 654614acc821..7580da77a138 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java @@ -35,7 +35,7 @@ public SinglePrimitiveOutputPTransform(PTransform> transf } @Override - public PCollection apply(PInput input) { + public PCollection expand(PInput input) { try { PCollection collection = PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.BOUNDED); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java index bd0c6558e03a..01a4cba00d4f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java @@ -281,7 +281,7 @@ public Bound withoutValidation() { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { if (filepattern == null) { throw new IllegalStateException( "need to set the filepattern of an AvroIO.Read transform"); @@ -795,7 +795,7 @@ public Bound withMetadata(Map metadata) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { if (filenamePrefix == null) { throw new IllegalStateException( "need to set the filename prefix of an AvroIO.Write transform"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index 123dca8aae95..f2ef358c5165 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -84,7 +84,7 @@ public BoundedReadFromUnboundedSource withMaxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { PCollection> read = Pipeline.applyTransform(input, Read.from(new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime))); if (source.requiresDeduping()) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java index 456d291ef35a..3148d8dd74d3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java @@ -130,7 +130,7 @@ private BoundedCountingInput(long startIndex, long endIndex) { } @Override - public PCollection apply(PBegin begin) { + public PCollection expand(PBegin begin) { return begin.apply(Read.from(CountingSource.createSourceForSubrange(startIndex, endIndex))); } @@ -236,7 +236,7 @@ public UnboundedCountingInput withMaxReadTime(Duration readTime) { @SuppressWarnings("deprecation") @Override - public PCollection apply(PBegin begin) { + public PCollection expand(PBegin begin) { Unbounded read = Read.from( CountingSource.createUnbounded() diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index 9768788ef7a5..9a6b5348e503 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -700,7 +700,7 @@ public Bound maxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { if (topic == null && subscription == null) { throw new IllegalStateException("Need to set either the topic or the subscription for " + "a PubsubIO.Read transform"); @@ -1057,7 +1057,7 @@ public Bound withCoder(Coder coder) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { if (topic == null) { throw new IllegalStateException("need to set the topic of a PubsubIO.Write transform"); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java index 1a86a1c88851..1992cb8f535a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSink.java @@ -424,7 +424,7 @@ public Coder getElementCoder() { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply("PubsubUnboundedSink.Window", Window.into(new GlobalWindows()) .triggering( Repeatedly.forever( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java index cea74bce583b..da3b4375ae71 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubUnboundedSource.java @@ -1339,7 +1339,7 @@ public String getIdLabel() { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { return input.getPipeline().begin() .apply(Read.from(new PubsubSource(this))) .apply("PubsubUnboundedSource.Stats", diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index f04fbaffb4fc..7ec3b0edd5ff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -101,7 +101,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PBegin input) { + public final PCollection expand(PBegin input) { source.validate(); return PCollection.createPrimitiveOutputInternal(input.getPipeline(), @@ -169,7 +169,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PBegin input) { + public final PCollection expand(PBegin input) { source.validate(); return PCollection.createPrimitiveOutputInternal( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index e967a27d3fd1..54e73d5cceb9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -292,7 +292,7 @@ public Bound withCompressionType(TextIO.CompressionType compressionType) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { if (filepattern == null) { throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform"); } @@ -742,7 +742,7 @@ public Bound withWritableByteChannelFactory( } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { if (filenamePrefix == null) { throw new IllegalStateException( "need to set the filename prefix of a TextIO.Write transform"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index 7559fca33a46..bc651d8d2769 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -104,7 +104,7 @@ private Bound(Sink sink, int numShards) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { checkArgument(IsBounded.BOUNDED == input.isBounded(), "%s can only be applied to a Bounded PCollection", Write.class.getSimpleName()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java index 77f51285f9eb..8604dbc65c48 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java @@ -73,6 +73,6 @@ public static PipelineRunner fromOptions(PipelineOptio */ public OutputT apply( PTransform transform, InputT input) { - return transform.apply(input); + return transform.expand(input); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java index 2b311b79a715..bf2cd0b3c503 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java @@ -55,7 +55,7 @@ public static GatherAllPanes globally() { private GatherAllPanes() {} @Override - public PCollection>> apply(PCollection input) { + public PCollection>> expand(PCollection input) { WindowFn originalWindowFn = input.getWindowingStrategy().getWindowFn(); return input 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 7dc78d86d61c..b23f4f3e69fd 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 @@ -763,7 +763,7 @@ private CreateActual( } @Override - public PCollectionView apply(PBegin input) { + public PCollectionView expand(PBegin input) { final Coder coder = actual.getCoder(); return actual .apply("FilterActuals", rewindowActuals.prepareActuals()) @@ -833,7 +833,7 @@ public GroupGlobally(AssertionWindows rewindowingStrategy) { } @Override - public PCollection>> apply(PCollection input) { + public PCollection>> expand(PCollection input) { final int combinedKey = 42; // Remove the triggering on both @@ -925,7 +925,7 @@ private GroupThenAssert( } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input .apply("GroupGlobally", new GroupGlobally(rewindowingStrategy)) .apply("GetPane", MapElements.via(paneExtractor)) @@ -958,7 +958,7 @@ private GroupThenAssertForSingleton( } @Override - public PDone apply(PCollection> input) { + public PDone expand(PCollection> input) { input .apply("GroupGlobally", new GroupGlobally>(rewindowingStrategy)) .apply("GetPane", MapElements.via(paneExtractor)) @@ -995,7 +995,7 @@ private OneSideInputAssert( } @Override - public PDone apply(PBegin input) { + public PDone expand(PBegin input) { final PCollectionView actual = input.apply("CreateActual", createActual); input @@ -1321,7 +1321,7 @@ public FilterWindows(StaticWindows windows) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply("FilterWindows", ParDo.of(new Fn())); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java index 509bb2481fbb..da93cdc9736b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -252,7 +252,7 @@ static Event advanceBy(Duration amount) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { throw new IllegalStateException( String.format( "Pipeline Runner %s does not provide a required override for %s", diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java index 71c2158fa42c..33820e05d540 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java @@ -204,7 +204,7 @@ public Globally(double maximumEstimationError) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { Coder coder = input.getCoder(); return input.apply( Combine.globally( @@ -271,7 +271,7 @@ public PerKey(double estimationError) { } @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { Coder> inputCoder = input.getCoder(); if (!(inputCoder instanceof KvCoder)) { throw new IllegalStateException( 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 4127d948b957..3b07260f7131 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 @@ -1431,7 +1431,7 @@ public boolean isInsertDefault() { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { PCollection> withKeys = input .apply(WithKeys.of((Void) null)) .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())); @@ -1569,7 +1569,7 @@ private GloballyAsSingletonView( } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { Globally combineGlobally = Combine.globally(fn).withoutDefaults().withFanout(fanout); if (insertDefault) { @@ -1866,7 +1866,7 @@ public List> getSideInputs() { } @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { return input .apply(GroupByKey.create(fewKeys)) .apply(Combine.groupedValues(fn, fnDisplayData) @@ -1901,7 +1901,7 @@ private PerKeyWithHotKeyFanout(String name, } @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { return applyHelper(input); } @@ -2388,7 +2388,7 @@ public List> getSideInputs() { } @Override - public PCollection> apply( + public PCollection> expand( PCollection>> input) { PCollection> output = input.apply(ParDo.of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java index b393a303951d..9101996da6c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java @@ -103,7 +103,7 @@ public static class PerElement public PerElement() { } @Override - public PCollection> apply(PCollection input) { + public PCollection> expand(PCollection input) { return input .apply("Init", MapElements.via(new SimpleFunction>() { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index 7cd47117565d..a48136fdfe8a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -240,7 +240,7 @@ public Iterable getElements() { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { try { Coder coder = getDefaultOutputCoder(input); try { @@ -440,7 +440,7 @@ public TimestampedValues withCoder(Coder coder) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { try { Iterable rawElements = Iterables.transform( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java index fba428b0ebee..2d08cee932eb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java @@ -82,7 +82,7 @@ public static WithRepresentativeValues withRepresentativeValueF } @Override - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in .apply("CreateIndex", MapElements.via(new SimpleFunction>() { @Override @@ -121,7 +121,7 @@ private WithRepresentativeValues( } @Override - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { WithKeys withKeys = WithKeys.of(fn); if (representativeType != null) { withKeys = withKeys.withKeyType(representativeType); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java index 2d9bdee54068..a564999f8ec8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java @@ -201,7 +201,7 @@ Filter described(String description) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply(ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java index 4ef809f32cbf..c165f7f12929 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java @@ -129,7 +129,7 @@ private FlatMapElements( } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply( "FlatMap", ParDo.of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java index e612836ce4ad..3ef2e555677e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java @@ -105,7 +105,7 @@ public static class FlattenPCollectionList private FlattenPCollectionList() { } @Override - public PCollection apply(PCollectionList inputs) { + public PCollection expand(PCollectionList inputs) { WindowingStrategy windowingStrategy; IsBounded isBounded = IsBounded.BOUNDED; if (!inputs.getAll().isEmpty()) { @@ -163,7 +163,7 @@ public static class FlattenIterables extends PTransform>, PCollection> { @Override - public PCollection apply(PCollection> in) { + public PCollection expand(PCollection> in) { Coder> inCoder = in.getCoder(); if (!(inCoder instanceof IterableLikeCoder)) { throw new IllegalArgumentException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java index 1faac5950dc9..a339af74f514 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java @@ -216,7 +216,7 @@ public void validate(PCollection> input) { } @Override - public PCollection>> apply(PCollection> input) { + public PCollection>> expand(PCollection> input) { // This primitive operation groups by the combination of key and window, // merging windows as needed, using the windows assigned to the // key/value input elements and the window merge operation of the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java index 2405adf41e4b..c6f307de692a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Keys.java @@ -56,7 +56,7 @@ public static Keys create() { private Keys() { } @Override - public PCollection apply(PCollection> in) { + public PCollection expand(PCollection> in) { return in.apply("Keys", MapElements.via(new SimpleFunction, K>() { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java index 2b81ebfdf1de..dbe262b82d20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/KvSwap.java @@ -60,7 +60,7 @@ public static KvSwap create() { private KvSwap() { } @Override - public PCollection> apply(PCollection> in) { + public PCollection> expand(PCollection> in) { return in.apply("KvSwap", MapElements.via(new SimpleFunction, KV>() { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java index 83ccecae7589..9c2d7154e155 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java @@ -159,7 +159,7 @@ public static PTransform>, PCollection>> pe private static class Globally extends PTransform, PCollection> { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { Coder inputCoder = input.getCoder(); return input @@ -178,7 +178,7 @@ public void processElement(ProcessContext c) { private static class PerKey extends PTransform>, PCollection>> { @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { checkNotNull(input); checkArgument(input.getCoder() instanceof KvCoder, "Input specifiedCoder must be an instance of KvCoder, but was %s", input.getCoder()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java index c1090345b69a..421b2abb1a51 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java @@ -111,7 +111,7 @@ private MapElements(SimpleFunction fn, Class fnClass) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply( "Map", ParDo.of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java index 83fe577a06da..ce4891dfba95 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java @@ -129,7 +129,7 @@ * output value as their apply implementation. * The majority of PTransforms are * implemented as composites of other PTransforms. Such a PTransform - * subclass typically just implements {@link #apply}, computing its + * subclass typically just implements {@link #expand}, computing its * Output value from its {@code InputT} value. User programs are encouraged to * use this mechanism to modularize their own code. Such composite * abstractions get their own name, and navigating through the @@ -181,7 +181,7 @@ public abstract class PTransform * a new unbound output and register evaluators (via backend-specific * registration methods). */ - public abstract OutputT apply(InputT input); + public abstract OutputT expand(InputT input); /** * Called before invoking apply (which may be intercepted by the runner) to diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index ba6e64423702..4f7491e008f6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -795,7 +795,7 @@ public BoundMulti withOutputTags( } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { checkArgument( !isSplittable(getOldFn()), "Splittable DoFn not supported by the current runner"); validateWindowType(input, fn); @@ -1052,7 +1052,7 @@ public BoundMulti withSideInputs( @Override - public PCollectionTuple apply(PCollection input) { + public PCollectionTuple expand(PCollection input) { checkArgument( !isSplittable(getOldFn()), "Splittable DoFn not supported by the current runner"); validateWindowType(input, fn); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java index b52587227b7f..e0b2b61fd7e9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java @@ -100,7 +100,7 @@ public static Partition of( ///////////////////////////////////////////////////////////////////////////// @Override - public PCollectionList apply(PCollection in) { + public PCollectionList expand(PCollection in) { final TupleTagList outputTags = partitionDoFn.getOutputTags(); PCollectionTuple outputs = in.apply( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java index a94130d38a92..14c5d1b6631b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java @@ -179,7 +179,7 @@ public Matches(String regex, int group) { this.group = group; } - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in.apply( ParDo.of( new DoFn() { @@ -224,7 +224,7 @@ public MatchesKV(String regex, int keyGroup, int valueGroup) { this.valueGroup = valueGroup; } - public PCollection> apply(PCollection in) { + public PCollection> expand(PCollection in) { return in.apply( ParDo.of( new DoFn>() { @@ -266,7 +266,7 @@ public Find(String regex, int group) { this.group = group; } - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in.apply( ParDo.of( new DoFn() { @@ -312,7 +312,7 @@ public FindKV(String regex, int keyGroup, int valueGroup) { this.valueGroup = valueGroup; } - public PCollection> apply(PCollection in) { + public PCollection> expand(PCollection in) { return in.apply( ParDo.of( new DoFn>() { @@ -354,7 +354,7 @@ public ReplaceAll(String regex, String replacement) { this.replacement = replacement; } - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in.apply( ParDo.of( new DoFn() { @@ -393,7 +393,7 @@ public ReplaceFirst(String regex, String replacement) { this.replacement = replacement; } - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in.apply( ParDo.of( new DoFn() { @@ -434,7 +434,7 @@ public Split(String regex, boolean outputEmpty) { this.outputEmpty = outputEmpty; } - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in.apply( ParDo.of( new DoFn() { 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 eca987a94227..7d4e63049aae 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 @@ -143,7 +143,7 @@ private SampleAny(long limit) { } @Override - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { PCollectionView> iterableView = in.apply(View.asIterable()); return in.getPipeline() diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java index d21d100764a0..faf402caf055 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Values.java @@ -56,7 +56,7 @@ public static Values create() { private Values() { } @Override - public PCollection apply(PCollection> in) { + public PCollection expand(PCollection> in) { return in.apply("Values", MapElements.via(new SimpleFunction, V>() { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java index 5fafc0a1ec23..126679d179fd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java @@ -251,7 +251,7 @@ public void validate(PCollection input) { } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { return input.apply(CreatePCollectionView.>of(PCollectionViews.listView( input.getPipeline(), input.getWindowingStrategy(), input.getCoder()))); } @@ -277,7 +277,7 @@ public void validate(PCollection input) { } @Override - public PCollectionView> apply(PCollection input) { + public PCollectionView> expand(PCollection input) { return input.apply(CreatePCollectionView.>of(PCollectionViews.iterableView( input.getPipeline(), input.getWindowingStrategy(), input.getCoder()))); } @@ -334,7 +334,7 @@ public void validate(PCollection input) { } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { return input.apply(CreatePCollectionView.of(PCollectionViews.singletonView( input.getPipeline(), input.getWindowingStrategy(), @@ -364,7 +364,7 @@ public void validate(PCollection> input) { } @Override - public PCollectionView>> apply(PCollection> input) { + public PCollectionView>> expand(PCollection> input) { return input.apply(CreatePCollectionView., Map>>of( PCollectionViews.multimapView( input.getPipeline(), @@ -401,7 +401,7 @@ public void validate(PCollection> input) { } @Override - public PCollectionView> apply(PCollection> input) { + public PCollectionView> expand(PCollection> input) { return input.apply(CreatePCollectionView., Map>of( PCollectionViews.mapView( input.getPipeline(), @@ -439,7 +439,7 @@ public PCollectionView getView() { } @Override - public PCollectionView apply(PCollection input) { + public PCollectionView expand(PCollection input) { return view; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java index de28ecba6afc..dd38006459d8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java @@ -111,7 +111,7 @@ public WithKeys withKeyType(TypeDescriptor keyType) { } @Override - public PCollection> apply(PCollection in) { + public PCollection> expand(PCollection in) { PCollection> result = in.apply("AddKeys", MapElements.via(new SimpleFunction>() { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java index 64e7c450dfa3..387707b54468 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithTimestamps.java @@ -98,7 +98,7 @@ public Duration getAllowedTimestampSkew() { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply( "AddTimestamps", ParDo.of(new AddTimestampsDoFn(fn, allowedTimestampSkew))); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java index b760e2c1a3ad..1d10da4dc3cb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGroupByKey.java @@ -87,7 +87,7 @@ public static CoGroupByKey create() { private CoGroupByKey() { } @Override - public PCollection> apply( + public PCollection> expand( KeyedPCollectionTuple input) { if (input.isEmpty()) { throw new IllegalArgumentException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index 5607762829ef..0c430d0ddc50 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -445,7 +445,7 @@ public void validate(PCollection input) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { WindowingStrategy outputStrategy = getOutputStrategyInternal(input.getWindowingStrategy()); return PCollection.createPrimitiveOutputInternal( @@ -517,7 +517,7 @@ public static Remerge remerge() { */ private static class Remerge extends PTransform, PCollection> { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { WindowingStrategy outputWindowingStrategy = getOutputWindowing( input.getWindowingStrategy()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java index ebd44bf4ab03..4d86c740634a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java @@ -50,7 +50,7 @@ public static Reshuffle of() { } @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { WindowingStrategy originalStrategy = input.getWindowingStrategy(); // If the input has already had its windows merged, then the GBK that performed the merge // will have set originalStrategy.getWindowFn() to InvalidWindows, causing the GBK contained diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index 0c743f7f2983..fea1554376b0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -212,7 +212,7 @@ public void testIdentityTransform() throws Exception { private static class IdentityTransform extends PTransform { @Override - public T apply(T input) { + public T expand(T input) { return input; } } @@ -247,7 +247,7 @@ public TupleProjectionTransform(TupleTag tag) { } @Override - public PCollection apply(PCollectionTuple input) { + public PCollection expand(PCollectionTuple input) { return input.get(tag); } } @@ -281,7 +281,7 @@ public TupleInjectionTransform(TupleTag tag) { } @Override - public PCollectionTuple apply(PCollection input) { + public PCollectionTuple expand(PCollection input) { return PCollectionTuple.of(tag, input); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index 530d75562f67..d7badab10d49 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -370,7 +370,7 @@ public void processElement(ProcessContext c) { } @Override public PCollection>> - apply(PCollection input) { + expand(PCollection input) { return input.apply(ParDo.of(new OutputDoFn())); } } @@ -435,7 +435,7 @@ public void processElement(ProcessContext c) { } @Override public PCollection>> - apply(PCollection input) { + expand(PCollection input) { return input.apply(ParDo.of(new OutputDoFn())); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index 5be5ff1e2b62..5a7c994d2871 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -123,7 +123,7 @@ public void processElement(ProcessContext c) { } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input .apply(window) .apply(ParDo.of(new AddArbitraryKey())) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index ea4318828f4c..b0c17d8c5aaf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -98,7 +98,7 @@ public void emptyCompositeSucceeds() { pcList, new PTransform, PCollection>() { @Override - public PCollection apply(PCollectionList input) { + public PCollection expand(PCollectionList input) { return input.get(0); } }); @@ -130,7 +130,7 @@ public void producingOwnAndOthersOutputsFails() { pcList, new PTransform, PCollectionList>() { @Override - public PCollectionList apply(PCollectionList input) { + public PCollectionList expand(PCollectionList input) { return appended; } }); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java index a81fb1ae7b2e..d70aa2f64c77 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java @@ -73,7 +73,7 @@ private static class InvalidCompositeTransform extends PTransform> { @Override - public PCollectionList apply(PBegin b) { + public PCollectionList expand(PBegin b) { // Composite transform: apply delegates to other transformations, // here a Create transform. PCollection result = b.apply(Create.of("hello", "world")); @@ -95,7 +95,7 @@ private static class UnboundOutputCreator extends PTransform, PDone> { @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { // Apply an operation so that this is a composite transform. input.apply(Count.perElement()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java index 311c8de369d4..bb2877ec5eb9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java @@ -181,7 +181,7 @@ static class VoidValues extends PTransform>, PCollection>> { @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { return input.apply(FlatMapElements., KV>via( new SimpleFunction, Iterable>>() { @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index 535ffec76bec..ebde11052cd3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -306,7 +306,7 @@ public void testGroupByKeyDirectUnbounded() { p.apply( new PTransform>>() { @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { return PCollection.>createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java index 4a34c57b58b6..ac3444b3ae89 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java @@ -271,7 +271,7 @@ static class VoidValues extends PTransform>, PCollection>> { @Override - public PCollection> apply(PCollection> input) { + public PCollection> expand(PCollection> input) { return input.apply(MapElements.via( new SimpleFunction, KV>() { @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java index bfe8225c9dc9..2b7caee650ea 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java @@ -36,7 +36,7 @@ public void testPopulateDisplayDataDefaultBehavior() { PTransform, PCollection> transform = new PTransform, PCollection>() { @Override - public PCollection apply(PCollection begin) { + public PCollection expand(PCollection begin) { throw new IllegalArgumentException("Should never be applied"); } }; 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 975507660080..41e795e0e84b 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 @@ -272,7 +272,7 @@ static class MultiFilter private static final TupleTag BY3 = new TupleTag("by3"){}; @Override - public PCollectionTuple apply(PCollection input) { + public PCollectionTuple expand(PCollection input) { PCollection by2 = input.apply("Filter2s", ParDo.of(new FilterFn(2))); PCollection by3 = input.apply("Filter3s", ParDo.of(new FilterFn(3))); return PCollectionTuple.of(BY2, by2).and(BY3, by3); @@ -840,7 +840,7 @@ public void testParDoInCustomTransform() { .apply(Create.of(inputs)) .apply("CustomTransform", new PTransform, PCollection>() { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply(ParDo.of(new TestDoFn())); } }); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index fe2d1251295a..3bf63fdbec63 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -1360,7 +1360,7 @@ private void testViewUnbounded( .apply( new PTransform>>() { @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { return PCollection.>createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java index 7630779bb3e1..feff3339161b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java @@ -46,7 +46,7 @@ public void testCompositeTransform() { PTransform, ? super POutput> myTransform = new PTransform, POutput> () { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply(ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) throws Exception { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java index f7f8d4012aa6..f892153b69e1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java @@ -78,7 +78,7 @@ public void testHasType() { DisplayData data = DisplayData.from(new PTransform, PCollection>() { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { throw new IllegalArgumentException("Should never be applied"); } @@ -133,7 +133,7 @@ public void testHasNamespace() { assertFalse(matcher.matches(DisplayData.from( new PTransform, PCollection>(){ @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { throw new IllegalArgumentException("Should never be applied"); } }))); @@ -206,7 +206,7 @@ static class SampleTransform extends PTransform, PCollection } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { throw new IllegalArgumentException("Should never be applied"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index fccd03138759..f5c1e73fd76e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -111,7 +111,7 @@ public void populateDisplayData(DisplayData.Builder builder) { Instant startTime = defaultStartTime; @Override - public PCollection apply(PCollection begin) { + public PCollection expand(PCollection begin) { throw new IllegalArgumentException("Should never be applied"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java index ab208dd57cc3..d4fab17fb244 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java @@ -70,7 +70,7 @@ public WindowedCount(WindowFn windowFn) { this.windowFn = windowFn; } @Override - public PCollection apply(PCollection in) { + public PCollection expand(PCollection in) { return in.apply("Window", Window.into(windowFn) .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java index 042e9e3d76ec..1ac176b10631 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java @@ -69,13 +69,13 @@ private EmbeddedOldDoFn getEmbedded() { private class EmbeddedPTransform extends PTransform { @Override - public PDone apply(PBegin begin) { + public PDone expand(PBegin begin) { throw new IllegalArgumentException("Should never be applied"); } private class Bound extends PTransform { @Override - public PDone apply(PBegin begin) { + public PDone expand(PBegin begin) { throw new IllegalArgumentException("Should never be applied"); } } @@ -129,7 +129,7 @@ public void testPTransformNameWithAnonOuterClass() throws Exception { AnonymousClass anonymousClassObj = new AnonymousClass() { class NamedInnerClass extends PTransform { @Override - public PDone apply(PBegin begin) { + public PDone expand(PBegin begin) { throw new IllegalArgumentException("Should never be applied"); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java index 4000e5db0276..e5f2019a19e9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java @@ -48,7 +48,7 @@ public class PDoneTest { */ static class EmptyTransform extends PTransform { @Override - public PDone apply(PBegin begin) { + public PDone expand(PBegin begin) { return PDone.in(begin.getPipeline()); } } @@ -64,7 +64,7 @@ public SimpleTransform(String filename) { } @Override - public PDone apply(PBegin begin) { + public PDone expand(PBegin begin) { return begin .apply(Create.of(LINES)) diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/SortValues.java b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/SortValues.java index 3bd9afaec367..d1b4d07537fd 100644 --- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/SortValues.java +++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/SortValues.java @@ -74,7 +74,7 @@ SortValues create( } @Override - public PCollection>>> apply( + public PCollection>>> expand( PCollection>>> input) { return input.apply( ParDo.of( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 8bfbd5355eff..f99ca7869613 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -631,7 +631,7 @@ public void validate(PBegin input) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { String uuid = randomUUIDString(); final String jobIdToken = "beam_job_" + uuid; @@ -840,7 +840,7 @@ static class PassThroughThenCleanup extends PTransform, PColle } @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { TupleTag mainOutput = new TupleTag<>(); TupleTag cleanupSignal = new TupleTag<>(); PCollectionTuple outputs = input.apply(ParDo.of(new IdentityFn()) @@ -1846,7 +1846,7 @@ public void validate(PCollection input) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { Pipeline p = input.getPipeline(); BigQueryOptions options = p.getOptions().as(BigQueryOptions.class); BigQueryServices bqServices = getBigQueryServices(); @@ -2803,7 +2803,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { // A naive implementation would be to simply stream data directly to BigQuery. // However, this could occasionally lead to duplicated data, e.g., when // a VM that runs this code is restarted and the code is re-run. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 1ee9253b67be..a83784babbe0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -286,7 +286,7 @@ public String getTableId() { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { BigtableSource source = new BigtableSource(new SerializableFunction() { @Override @@ -500,7 +500,7 @@ public String getTableId() { } @Override - public PDone apply(PCollection>> input) { + public PDone expand(PCollection>> input) { input.apply(ParDo.of(new BigtableWriterFn(tableId, new SerializableFunction() { @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index d1a9a67d2c1d..1e8271caf887 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -411,7 +411,7 @@ public DatastoreV1.Read withNumQuerySplits(int numQuerySplits) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { V1Options v1Options = V1Options.from(getProjectId(), getQuery(), getNamespace()); @@ -779,7 +779,7 @@ private abstract static class Mutate extends PTransform, PDone } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply("Convert to Mutation", MapElements.via(mutationFn)) .apply("Write Mutation to Datastore", ParDo.of(new DatastoreWriterFn(projectId))); diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index 9644a65f96ab..c625287772e3 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -296,7 +296,7 @@ public Read withCoder(Coder coder) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { return input .apply(Create.of(getQuery())) .apply(ParDo.of(new ReadFn<>(this))).setCoder(getCoder()) @@ -417,7 +417,7 @@ public Write withPreparedStatementSetter(PreparedStatementSetter setter) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply(ParDo.of(new WriteFn(this))); return PDone.in(input.getPipeline()); } diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 00b91ad530c5..24fa67dbdda3 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -135,7 +135,7 @@ public Read withMaxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { // handles unbounded source to bounded conversion if maxNumRecords is set. Unbounded unbounded = org.apache.beam.sdk.io.Read.from(createSource()); @@ -434,7 +434,7 @@ private Write(ConnectionFactory connectionFactory, String queue, String topic) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply(ParDo.of(new JmsWriter(connectionFactory, queue, topic))); return PDone.in(input.getPipeline()); } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index c87d12b8dac8..735b8e7ac4fc 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -445,7 +445,7 @@ public PTransform>> withoutMetadata() { } @Override - public PCollection> apply(PBegin input) { + public PCollection> expand(PBegin input) { // Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set. Unbounded> unbounded = org.apache.beam.sdk.io.Read.from(makeSource()); @@ -544,9 +544,9 @@ public static class TypedWithoutMetadata extends PTransform> apply(PBegin begin) { + public PCollection> expand(PBegin begin) { return typedRead - .apply(begin) + .expand(begin) .apply("Remove Kafka Metadata", ParDo.of(new DoFn, KV>() { @ProcessElement @@ -1244,7 +1244,7 @@ public PTransform, PDone> values() { } @Override - public PDone apply(PCollection> input) { + public PDone expand(PCollection> input) { input.apply(ParDo.of(new KafkaWriter( topic, keyCoder, valueCoder, producerConfig, producerFactoryFnOpt))); return PDone.in(input.getPipeline()); @@ -1311,7 +1311,7 @@ private KafkaValueWrite(TypedWrite kvWriteTransform) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { return input .apply("Kafka values with default key", MapElements.via(new SimpleFunction>() { diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java index a482df46149f..b65e6710a8c1 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java @@ -308,7 +308,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { final BoundedGridFSSource source = new BoundedGridFSSource(this, null); org.apache.beam.sdk.io.Read.Bounded objectIds = org.apache.beam.sdk.io.Read.from(source); @@ -621,7 +621,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply(ParDo.of(new GridFsWriteFn(this))); return PDone.in(input.getPipeline()); } diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index 70239e672a8e..f5394319efb7 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -160,7 +160,7 @@ public Read withNumSplits(int numSplits) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { return input.apply(org.apache.beam.sdk.io.Read.from(new BoundedMongoDbSource(this))); } @@ -441,7 +441,7 @@ public Write withBatchSize(long batchSize) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply(ParDo.of(new WriteFn(this))); return PDone.in(input.getPipeline()); } From 96c6a3b4cd46c6109ecc532b6e5f7fd47f702c9e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:49:15 -0800 Subject: [PATCH 116/279] Add JUnit category UsesTimersInParDo With this, various runners can disable tests for this capability until they support it. --- .../beam/sdk/testing/UsesTimersInParDo.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimersInParDo.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimersInParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimersInParDo.java new file mode 100644 index 000000000000..14123ed6a8d3 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimersInParDo.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; + +import org.apache.beam.sdk.transforms.ParDo; + +/** + * Category tag for validation tests which utilize timers in {@link ParDo}. + */ +public interface UsesTimersInParDo {} From 54c14be11549f5733dc43932b08096b33ff74af4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:52:24 -0800 Subject: [PATCH 117/279] Disable tests for timers in ParDo for Apex runner --- runners/apex/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 629e89080803..b60423757e16 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -187,6 +187,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo none From 2f13fe412c20f21b711b7686cdd18c68efbdd038 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:52:49 -0800 Subject: [PATCH 118/279] Disable tests for timers in ParDo for Flink runner --- runners/flink/runner/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 615d5f1eca59..9509476a0428 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -55,6 +55,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo none From 320f8881e91628fa843e0eb2918d2de7a7aae329 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:53:05 -0800 Subject: [PATCH 119/279] Disables tests for timers in ParDo for Spark runner --- runners/spark/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 9a3adf6af93a..d1ef225ceeb5 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -74,6 +74,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo 1 From 5a8f9565d82dd813428f08d97f975c69b28a58ee Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:53:16 -0800 Subject: [PATCH 120/279] Disable tests for timers in ParDo for Dataflow runner --- runners/google-cloud-dataflow-java/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 9ead74a17d6a..1543c0e00937 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -78,6 +78,7 @@ runnable-on-service-tests + org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo From a99dba5955813a092e78fb438a1b5b96480fae3a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:24:34 -0800 Subject: [PATCH 121/279] Disable tests for timers in ParDo for direct runner --- runners/direct-java/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 79f3de61a5c1..c47f53270576 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -68,6 +68,7 @@ org.apache.beam.sdk.testing.NeedsRunner + org.apache.beam.sdk.testing.UsesTimersInParDo none true From 50ffc7be7f41a38ce214f30dd76aa56ddbd245aa Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Dec 2016 20:49:40 -0800 Subject: [PATCH 122/279] Add basic test for timers in ParDoTest --- .../apache/beam/sdk/transforms/ParDoTest.java | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) 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 41e795e0e84b..36666b24423b 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 @@ -55,7 +55,9 @@ 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.TestStream; import org.apache.beam.sdk.testing.UsesStatefulParDo; +import org.apache.beam.sdk.testing.UsesTimersInParDo; import org.apache.beam.sdk.transforms.DoFn.OnTimer; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo.Bound; @@ -69,6 +71,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; @@ -1571,6 +1574,52 @@ public void processElement( p.run(); } + /** + * Tests that an event time timer fires and results in supplementary output. + * + *

      This test relies on two properties: + * + *

        + *
      1. A timer that is set on time should always get a chance to fire. For this to be true, timers + * per-key-and-window must be delivered in order so the timer is not wiped out until the + * window is expired by the runner. + *
      2. A {@link Create} transform sends its elements on time, and later advances the watermark to + * infinity + *
      + * + *

      Note that {@link TestStream} is not applicable because it requires very special runner hooks + * and is only supported by the direct runner. + */ + @Test + @Category({RunnableOnService.class, UsesTimersInParDo.class}) + public void testSimpleEventTimeTimer() throws Exception { + final String timerId = "foo"; + + DoFn, Integer> fn = + new DoFn, Integer>() { + + @TimerId(timerId) + private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { + timer.setForNowPlus(Duration.standardSeconds(1)); + context.output(3); + } + + @OnTimer(timerId) + public void onTimer(OnTimerContext context) { + context.output(42); + } + }; + + Pipeline p = TestPipeline.create(); + + PCollection output = p.apply(Create.of(KV.of("hello", 37))).apply(ParDo.of(fn)); + PAssert.that(output).containsInAnyOrder(3, 42); + p.run(); + } + @Test public void testWithOutputTagsDisplayData() { DoFn fn = new DoFn() { From 18db3ace77e89203d7ec3f342fe6ce24a2119226 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:34:34 -0800 Subject: [PATCH 123/279] Reject timers for ParDo in ApexRunner --- .../translation/ParDoBoundMultiTranslator.java | 16 +++++++++++++++- .../apex/translation/ParDoBoundTranslator.java | 16 +++++++++++++++- 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java index fed5f4b37480..706482a5d07f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; @@ -54,7 +55,9 @@ class ParDoBoundMultiTranslator @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.stateDeclarations().size() > 0) { throw new UnsupportedOperationException( String.format( "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", @@ -63,6 +66,17 @@ public void translate(ParDo.BoundMulti transform, TranslationCo DoFn.class.getSimpleName(), ApexRunner.class.getSimpleName())); } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + ApexRunner.class.getSimpleName())); + } + OldDoFn oldDoFn = transform.getFn(); PCollectionTuple output = context.getOutput(); PCollection input = context.getInput(); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java index 7a918a71e431..b5a50f671fd2 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; @@ -41,7 +42,9 @@ class ParDoBoundTranslator @Override public void translate(ParDo.Bound transform, TranslationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.stateDeclarations().size() > 0) { throw new UnsupportedOperationException( String.format( "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", @@ -50,6 +53,17 @@ public void translate(ParDo.Bound transform, TranslationContext DoFn.class.getSimpleName(), ApexRunner.class.getSimpleName())); } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + ApexRunner.class.getSimpleName())); + } + OldDoFn oldDoFn = transform.getOldFn(); PCollection output = context.getOutput(); PCollection input = context.getInput(); From 69e0ea25f24597b84c93137dd94e2f25a9b88a15 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:34:59 -0800 Subject: [PATCH 124/279] Reject timers for ParDo in FlinkRunner --- .../FlinkBatchTransformTranslators.java | 46 +++++++++++-------- .../FlinkStreamingTransformTranslators.java | 45 ++++++++++-------- 2 files changed, 54 insertions(+), 37 deletions(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index 474d4e397cfb..9ac907f6eb9d 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 @@ -56,6 +56,7 @@ import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.join.UnionCoder; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -483,6 +484,30 @@ public void translateNode( } } + private static void rejectStateAndTimers(DoFn doFn) { + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + } + private static class ParDoBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator< ParDo.Bound> { @@ -493,15 +518,7 @@ public void translateNode( FlinkBatchTranslationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - FlinkRunner.class.getSimpleName())); - } + rejectStateAndTimers(doFn); DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); @@ -549,16 +566,7 @@ public void translateNode( ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - FlinkRunner.class.getSimpleName())); - } - + rejectStateAndTimers(doFn); DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); 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 7b32c7649a38..042f8dfc5a6f 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 @@ -57,6 +57,7 @@ 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.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -304,6 +305,30 @@ public void translateNode( } } + private static void rejectStateAndTimers(DoFn doFn) { + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + FlinkRunner.class.getSimpleName())); + } + } + private static class ParDoBoundStreamingTranslator extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< ParDo.Bound> { @@ -314,15 +339,7 @@ public void translateNode( FlinkStreamingTranslationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - FlinkRunner.class.getSimpleName())); - } + rejectStateAndTimers(doFn); WindowingStrategy windowingStrategy = context.getOutput(transform).getWindowingStrategy(); @@ -474,15 +491,7 @@ public void translateNode( FlinkStreamingTranslationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - FlinkRunner.class.getSimpleName())); - } + rejectStateAndTimers(doFn); // we assume that the transformation does not change the windowing strategy. WindowingStrategy windowingStrategy = From 29f3af30a4b871244e14998d670b8ca26bd8de94 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:35:08 -0800 Subject: [PATCH 125/279] Reject timers for ParDo in SparkRunner --- .../translation/TransformTranslator.java | 46 +++++++++++-------- 1 file changed, 27 insertions(+), 19 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index e033ab1255d7..8170366614b4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -58,6 +58,7 @@ 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.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; @@ -81,7 +82,6 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; - import scala.Tuple2; @@ -228,20 +228,36 @@ public void evaluate(Combine.PerKey transform, }; } + private static void rejectStateAndTimers(DoFn doFn) { + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + SparkRunner.class.getSimpleName())); + } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + SparkRunner.class.getSimpleName())); + } + } + private static TransformEvaluator> parDo() { return new TransformEvaluator>() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - SparkRunner.class.getSimpleName())); - } + rejectStateAndTimers(doFn); @SuppressWarnings("unchecked") JavaRDD> inRDD = ((BoundedDataset) context.borrowDataset(transform)).getRDD(); @@ -265,15 +281,7 @@ public void evaluate(ParDo.Bound transform, EvaluationContext c @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { DoFn doFn = transform.getNewFn(); - if (DoFnSignatures.getSignature(doFn.getClass()).stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - SparkRunner.class.getSimpleName())); - } + rejectStateAndTimers(doFn); @SuppressWarnings("unchecked") JavaRDD> inRDD = ((BoundedDataset) context.borrowDataset(transform)).getRDD(); From 274f17f0c0df08785a78d9a60c22d5556e46584a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:37:33 -0800 Subject: [PATCH 126/279] Reject timers for ParDo in DirectRunner --- .../runners/direct/ParDoMultiOverrideFactory.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 8c96e9b59d4f..4e7914f310eb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -50,9 +50,17 @@ public PTransform, PCollectionTuple> override( DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); if (signature.processElement().isSplittable()) { return new SplittableParDo(transform); - } else if (signature.stateDeclarations().size() > 0 - || signature.timerDeclarations().size() > 0) { - + } else if (signature.timerDeclarations().size() > 0) { + // Temporarily actually reject timers + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + fn.getClass().getName(), + DoFn.class.getSimpleName(), + DirectRunner.class.getSimpleName())); + + } else if (signature.stateDeclarations().size() > 0) { // Based on the fact that the signature is stateful, DoFnSignatures ensures // that it is also keyed ParDo.BoundMulti, OutputT> keyedTransform = From c22e2a435113c9653b58f1040a4e9266059767f4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:04:51 -0800 Subject: [PATCH 127/279] No longer reject timers in ParDo --- .../java/org/apache/beam/sdk/transforms/ParDo.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 4f7491e008f6..e60c536a9650 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -594,16 +594,6 @@ private static void validateWindowTypeForMethod( private static void validate(DoFn fn) { DoFnSignature signature = DoFnSignatures.getSignature((Class) fn.getClass()); - // To be removed when the features are complete and runners have their own adequate - // rejection logic - if (!signature.timerDeclarations().isEmpty()) { - throw new UnsupportedOperationException( - String.format("Found %s annotations on %s, but %s cannot yet be used with timers.", - DoFn.TimerId.class.getSimpleName(), - fn.getClass().getName(), - DoFn.class.getSimpleName())); - } - // State is semantically incompatible with splitting if (!signature.stateDeclarations().isEmpty() && signature.processElement().isSplittable()) { throw new UnsupportedOperationException( From 95e2c53db535952aaf0c335e0d3d27a721c6b55d Mon Sep 17 00:00:00 2001 From: Sela Date: Thu, 8 Dec 2016 20:29:35 +0200 Subject: [PATCH 128/279] [BEAM-1111] Reject timers for ParDo in SparkRunner streaming evaluators --- .../translation/TransformTranslator.java | 28 +--------------- .../spark/translation/TranslationUtils.java | 33 +++++++++++++++++++ .../StreamingTransformTranslator.java | 6 ++++ 3 files changed, 40 insertions(+), 27 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 8170366614b4..964eb3707fe0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -23,6 +23,7 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFilePrefix; import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate; import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount; +import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectStateAndTimers; import com.google.common.collect.Maps; import java.io.IOException; @@ -32,7 +33,6 @@ import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.beam.runners.core.AssignWindowsDoFn; -import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.runners.spark.aggregators.SparkAggregators; import org.apache.beam.runners.spark.coders.CoderHelpers; @@ -58,8 +58,6 @@ 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.reflect.DoFnSignature; -import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -228,30 +226,6 @@ public void evaluate(Combine.PerKey transform, }; } - private static void rejectStateAndTimers(DoFn doFn) { - DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); - - if (signature.stateDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", - DoFn.StateId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - SparkRunner.class.getSimpleName())); - } - - if (signature.timerDeclarations().size() > 0) { - throw new UnsupportedOperationException( - String.format( - "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", - DoFn.TimerId.class.getSimpleName(), - doFn.getClass().getName(), - DoFn.class.getSimpleName(), - SparkRunner.class.getSimpleName())); - } - } - private static TransformEvaluator> parDo() { return new TransformEvaluator>() { @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java index 647f8c3805a3..eddc771efd86 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java @@ -24,8 +24,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; @@ -211,4 +215,33 @@ public Boolean call(Tuple2, WindowedValue> input) { } } + /** + * Reject state and timers {@link DoFn}. + * + * @param doFn the {@link DoFn} to possibly reject. + */ + public static void rejectStateAndTimers(DoFn doFn) { + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); + + if (signature.stateDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with state in the %s.", + DoFn.StateId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + SparkRunner.class.getSimpleName())); + } + + if (signature.timerDeclarations().size() > 0) { + throw new UnsupportedOperationException( + String.format( + "Found %s annotations on %s, but %s cannot yet be used with timers in the %s.", + DoFn.TimerId.class.getSimpleName(), + doFn.getClass().getName(), + DoFn.class.getSimpleName(), + SparkRunner.class.getSimpleName())); + } + } + } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 85d796a71cc2..00df7d4a5951 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.spark.translation.streaming; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectStateAndTimers; import com.google.common.collect.Maps; import java.util.ArrayList; @@ -47,6 +48,7 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; +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.OldDoFn; @@ -348,6 +350,8 @@ private static TransformEvaluator @Override public void evaluate(final ParDo.Bound transform, final EvaluationContext context) { + DoFn doFn = transform.getNewFn(); + rejectStateAndTimers(doFn); final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); final Map, KV, BroadcastHelper>> sideInputs = TranslationUtils.getSideInputs(transform.getSideInputs(), context); @@ -380,6 +384,8 @@ public JavaRDD> call(JavaRDD> rdd) @Override public void evaluate(final ParDo.BoundMulti transform, final EvaluationContext context) { + DoFn doFn = transform.getNewFn(); + rejectStateAndTimers(doFn); final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); final Map, KV, BroadcastHelper>> sideInputs = TranslationUtils.getSideInputs(transform.getSideInputs(), context); From a11eb637f6f1c7bdc41b24154cb3893bb194b096 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 8 Dec 2016 12:42:03 -0800 Subject: [PATCH 129/279] Fix exclusion of UsesTimersInParDo for FlinkRunner streaming tests --- runners/flink/runner/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 9509476a0428..37de89962df9 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -85,6 +85,7 @@ org.apache.beam.sdk.testing.RunnableOnService org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo none From 4a12cd303df6d4c49425d38a6290f496349cc312 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 10 Nov 2016 21:53:57 -0800 Subject: [PATCH 130/279] [BEAM-978] Support bulk get file size in GcsUtil. --- .../org/apache/beam/sdk/util/GcsUtil.java | 90 +++++++++++---- .../org/apache/beam/sdk/util/GcsUtilTest.java | 106 +++++++++--------- 2 files changed, 121 insertions(+), 75 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java index 54a963b05a6f..2edb1d6932c8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java @@ -93,8 +93,10 @@ public static class GcsUtilFactory implements DefaultValueFactory { public GcsUtil create(PipelineOptions options) { LOG.debug("Creating new GcsUtil"); GcsOptions gcsOptions = options.as(GcsOptions.class); - return new GcsUtil(Transport.newStorageClient(gcsOptions).build(), - gcsOptions.getExecutorService(), gcsOptions.getGcsUploadBufferSizeBytes()); + return new GcsUtil( + Transport.newStorageClient(gcsOptions).build(), + gcsOptions.getExecutorService(), + gcsOptions.getGcsUploadBufferSizeBytes()); } } @@ -153,7 +155,8 @@ public boolean isGcsPatternSupported(String gcsPattern) { } private GcsUtil( - Storage storageClient, ExecutorService executorService, + Storage storageClient, + ExecutorService executorService, @Nullable Integer uploadBufferSizeBytes) { this.storageClient = storageClient; this.uploadBufferSizeBytes = uploadBufferSizeBytes; @@ -264,10 +267,7 @@ Integer getUploadBufferSizeBytes() { * if the resource does not exist. */ public long fileSize(GcsPath path) throws IOException { - return fileSize( - path, - BACKOFF_FACTORY.backoff(), - Sleeper.DEFAULT); + return fileSizes(ImmutableList.of(path)).get(0); } /** @@ -275,23 +275,15 @@ public long fileSize(GcsPath path) throws IOException { * if the resource does not exist. */ @VisibleForTesting - long fileSize(GcsPath path, BackOff backoff, Sleeper sleeper) throws IOException { - Storage.Objects.Get getObject = - storageClient.objects().get(path.getBucket(), path.getObject()); - try { - StorageObject object = ResilientOperation.retry( - ResilientOperation.getGoogleRequestCallable(getObject), - backoff, - RetryDeterminer.SOCKET_ERRORS, - IOException.class, - sleeper); - return object.getSize().longValue(); - } catch (Exception e) { - if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) { - throw new FileNotFoundException(path.toString()); - } - throw new IOException("Unable to get file size", e); - } + List fileSizes(Collection paths) throws IOException { + List results = Lists.newArrayList(); + executeBatches(makeGetBatches(paths, results)); + + ImmutableList.Builder ret = ImmutableList.builder(); + for (long[] result : results) { + ret.add(result[0]); + } + return ret.build(); } /** @@ -483,12 +475,39 @@ public Void call() throws IOException { Thread.currentThread().interrupt(); throw new IOException("Interrupted while executing batch GCS request", e); } catch (ExecutionException e) { + if (e.getCause() instanceof FileNotFoundException) { + throw (FileNotFoundException) e.getCause(); + } throw new IOException("Error executing batch GCS request", e); } finally { executor.shutdown(); } } + /** + * Makes get {@link BatchRequest BatchRequests}. + * + * @param paths {@link GcsPath GcsPaths}. + * @param results mutable {@link List} for return values. + * @return {@link BatchRequest BatchRequests} to execute. + * @throws IOException + */ + @VisibleForTesting + List makeGetBatches( + Collection paths, + List results) throws IOException { + List batches = new LinkedList<>(); + for (List filesToGet : + Lists.partition(Lists.newArrayList(paths), MAX_REQUESTS_PER_BATCH)) { + BatchRequest batch = storageClient.batch(); + for (GcsPath path : filesToGet) { + results.add(enqueueGetFileSize(path, batch)); + } + batches.add(batch); + } + return batches; + } + public void copy(List srcFilenames, List destFilenames) throws IOException { executeBatches(makeCopyBatches(srcFilenames, destFilenames)); } @@ -535,6 +554,29 @@ public void remove(Collection filenames) throws IOException { executeBatches(makeRemoveBatches(filenames)); } + private long[] enqueueGetFileSize(final GcsPath path, BatchRequest batch) throws IOException { + final long[] fileSize = new long[1]; + + Storage.Objects.Get getRequest = storageClient.objects() + .get(path.getBucket(), path.getObject()); + getRequest.queue(batch, new JsonBatchCallback() { + @Override + public void onSuccess(StorageObject response, HttpHeaders httpHeaders) throws IOException { + fileSize[0] = response.getSize().longValue(); + } + + @Override + public void onFailure(GoogleJsonError e, HttpHeaders httpHeaders) throws IOException { + if (errorExtractor.itemNotFound(e)) { + throw new FileNotFoundException(path.toString()); + } else { + throw new IOException(String.format("Error trying to get %s: %s", path, e)); + } + } + }); + return fileSize; + } + private void enqueueCopy(final GcsPath from, final GcsPath to, BatchRequest batch) throws IOException { Storage.Objects.Copy copyRequest = storageClient.objects() diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java index df3bf6e58fa2..c8ed402f3cfb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java @@ -54,9 +54,9 @@ import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel; import com.google.cloud.hadoop.util.ClientRequestHelper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import java.io.FileNotFoundException; import java.io.IOException; -import java.math.BigInteger; import java.net.SocketTimeoutException; import java.nio.channels.SeekableByteChannel; import java.nio.file.AccessDeniedException; @@ -320,65 +320,35 @@ public void testAccessDeniedObjectThrowsIOException() throws IOException { } @Test - public void testGetSizeBytes() throws Exception { - GcsOptions pipelineOptions = gcsOptionsWithTestCredential(); - GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); - - Storage mockStorage = Mockito.mock(Storage.class); - gcsUtil.setStorageClient(mockStorage); - - Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); - Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class); + public void testGetSizeBytesWhenFileNotFound() throws Exception { + JsonFactory jsonFactory = new JacksonFactory(); - when(mockStorage.objects()).thenReturn(mockStorageObjects); - when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet); - when(mockStorageGet.execute()).thenReturn( - new StorageObject().setSize(BigInteger.valueOf(1000))); + String contentBoundary = "batch_foobarbaz"; - assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"))); - } + GenericJson error = new GenericJson() + .set("error", new GenericJson().set("code", 404)); + error.setFactory(jsonFactory); - @Test - public void testGetSizeBytesWhenFileNotFound() throws Exception { - MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse(); - notFoundResponse.setContent(""); - notFoundResponse.setStatusCode(HttpStatusCodes.STATUS_CODE_NOT_FOUND); + String content = contentBoundary + "\n" + + "Content-Type: application/http\n" + + "\n" + + "HTTP/1.1 404 Not Found\n" + + "Content-Length: 105\n" + + "\n" + + error.toString(); + thrown.expect(FileNotFoundException.class); + MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse() + .setContentType("multipart/mixed; boundary=" + contentBoundary) + .setContent(content) + .setStatusCode(HttpStatusCodes.STATUS_CODE_OK); MockHttpTransport mockTransport = new MockHttpTransport.Builder().setLowLevelHttpResponse(notFoundResponse).build(); - GcsOptions pipelineOptions = gcsOptionsWithTestCredential(); - GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); gcsUtil.setStorageClient(new Storage(mockTransport, Transport.getJsonFactory(), null)); - - thrown.expect(FileNotFoundException.class); - gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject")); - } - - @Test - public void testRetryFileSize() throws IOException { - GcsOptions pipelineOptions = gcsOptionsWithTestCredential(); - GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); - - Storage mockStorage = Mockito.mock(Storage.class); - gcsUtil.setStorageClient(mockStorage); - - Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); - Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class); - - BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff(); - - when(mockStorage.objects()).thenReturn(mockStorageObjects); - when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet); - when(mockStorageGet.execute()) - .thenThrow(new SocketTimeoutException("SocketException")) - .thenThrow(new SocketTimeoutException("SocketException")) - .thenReturn(new StorageObject().setSize(BigInteger.valueOf(1000))); - - assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"), - mockBackOff, new FastNanoClockAndSleeper())); - assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis()); + gcsUtil.fileSizes(ImmutableList.of(GcsPath.fromComponents("testbucket", "testobject"))); } @Test @@ -599,6 +569,14 @@ private static List makeStrings(String s, int n) { return ret.build(); } + private static List makeGcsPaths(String s, int n) { + ImmutableList.Builder ret = ImmutableList.builder(); + for (int i = 0; i < n; ++i) { + ret.add(GcsPath.fromUri(String.format("gs://bucket/%s%d", s, i))); + } + return ret.build(); + } + private static int sumBatchSizes(List batches) { int ret = 0; for (BatchRequest b : batches) { @@ -656,4 +634,30 @@ public void testMakeRemoveBatches() throws IOException { assertThat(batches.size(), equalTo(6)); assertThat(sumBatchSizes(batches), equalTo(501)); } + + @Test + public void testMakeGetBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + + // Small number of files fits in 1 batch + List results = Lists.newArrayList(); + List batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 3), results); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(3)); + assertEquals(3, results.size()); + + // 1 batch of files fits in 1 batch + results = Lists.newArrayList(); + batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 100), results); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(100)); + assertEquals(100, results.size()); + + // A little more than 5 batches of files fits in 6 batches + results = Lists.newArrayList(); + batches = gcsUtil.makeGetBatches(makeGcsPaths("s", 501), results); + assertThat(batches.size(), equalTo(6)); + assertThat(sumBatchSizes(batches), equalTo(501)); + assertEquals(501, results.size()); + } } From bf1fba450e6b5fd6c98d006b381472eee8db7b72 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 6 Dec 2016 18:00:03 -0800 Subject: [PATCH 131/279] Fix a bug in SplittableDoFn Checkpointing Call checkpoint() only once if the SDF emits output several times per claim call. Calling checkpoint multiple times would clobber an existing checkpoint, and the second call would only ever return an empty residual, losing all of the initial residual. --- .../beam/runners/core/SplittableParDo.java | 9 ++- .../org/apache/beam/sdk/transforms/ParDo.java | 8 ++- .../apache/beam/sdk/transforms/ParDoTest.java | 6 +- .../sdk/transforms/SplittableDoFnTest.java | 58 ++++++++++++++++++- 4 files changed, 73 insertions(+), 8 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 0bf882bd13a9..8a9bfcd205f5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -590,9 +590,14 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { } private void noteOutput() { - if (++numOutputs >= MAX_OUTPUTS_PER_BUNDLE) { + // Take the checkpoint only if it hasn't been taken yet, because: + // 1) otherwise we'd lose the previous checkpoint stored in residualRestrictionHolder + // 2) it's not allowed to checkpoint a RestrictionTracker twice, since the first call + // by definition already maximally narrows its restriction, so a second checkpoint would + // have produced a useless empty residual restriction anyway. + if (++numOutputs >= MAX_OUTPUTS_PER_BUNDLE && residualRestrictionHolder[0] == null) { // Request a checkpoint. The fn *may* produce more output, but hopefully not too much. - residualRestrictionHolder[0] = tracker.checkpoint(); + residualRestrictionHolder[0] = checkNotNull(tracker.checkpoint()); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index e60c536a9650..167f5faad7e7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -787,7 +787,9 @@ public BoundMulti withOutputTags( @Override public PCollection expand(PCollection input) { checkArgument( - !isSplittable(getOldFn()), "Splittable DoFn not supported by the current runner"); + !isSplittable(getOldFn()), + "%s does not support Splittable DoFn", + input.getPipeline().getOptions().getRunner().getName()); validateWindowType(input, fn); return PCollection.createPrimitiveOutputInternal( input.getPipeline(), @@ -1044,7 +1046,9 @@ public BoundMulti withSideInputs( @Override public PCollectionTuple expand(PCollection input) { checkArgument( - !isSplittable(getOldFn()), "Splittable DoFn not supported by the current runner"); + !isSplittable(getOldFn()), + "%s does not support Splittable DoFn", + input.getPipeline().getOptions().getRunner().getName()); validateWindowType(input, fn); PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), 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 36666b24423b..2d118e44c0bc 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 @@ -1717,7 +1717,8 @@ public void testRejectsSplittableDoFnByDefault() { Pipeline p = TestPipeline.create(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Splittable DoFn not supported by the current runner"); + thrown.expectMessage(p.getRunner().getClass().getName()); + thrown.expectMessage("does not support Splittable DoFn"); p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new TestSplittableDoFn())); } @@ -1729,7 +1730,8 @@ public void testMultiRejectsSplittableDoFnByDefault() { Pipeline p = TestPipeline.create(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Splittable DoFn not supported by the current runner"); + thrown.expectMessage(p.getRunner().getClass().getName()); + thrown.expectMessage("does not support Splittable DoFn"); p.apply(Create.of(1, 2, 3)) .apply( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java index 82bd3a3c14c9..022c2e5cd4b4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.resume; +import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.stop; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -36,6 +38,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.UsesSplittableParDo; +import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -117,10 +120,10 @@ public ProcessContinuation process(ProcessContext c, OffsetRangeTracker tracker) for (int i = tracker.currentRestriction().from; tracker.tryClaim(i); ++i) { c.output(KV.of(c.element(), i)); if (i % 3 == 0) { - return ProcessContinuation.resume(); + return resume(); } } - return ProcessContinuation.stop(); + return stop(); } @GetInitialRestriction @@ -231,6 +234,57 @@ public void testPairWithIndexWindowedTimestamped() { p.run(); } + @BoundedPerElement + private static class SDFWithMultipleOutputsPerBlock extends DoFn { + private static final int MAX_INDEX = 98765; + + private static int snapToNextBlock(int index, int[] blockStarts) { + for (int i = 1; i < blockStarts.length; ++i) { + if (index > blockStarts[i - 1] && index <= blockStarts[i]) { + return i; + } + } + throw new IllegalStateException("Shouldn't get here"); + } + + @ProcessElement + public ProcessContinuation processElement(ProcessContext c, OffsetRangeTracker tracker) { + int[] blockStarts = {-1, 0, 12, 123, 1234, 12345, 34567, MAX_INDEX}; + int trueStart = snapToNextBlock(tracker.currentRestriction().from, blockStarts); + int trueEnd = snapToNextBlock(tracker.currentRestriction().to, blockStarts); + for (int i = trueStart; i < trueEnd; ++i) { + if (!tracker.tryClaim(blockStarts[i])) { + return resume(); + } + for (int index = blockStarts[i]; index < blockStarts[i + 1]; ++index) { + c.output(index); + } + } + return stop(); + } + + @GetInitialRestriction + public OffsetRange getInitialRange(String element) { + return new OffsetRange(0, MAX_INDEX); + } + + @NewTracker + public OffsetRangeTracker newTracker(OffsetRange range) { + return new OffsetRangeTracker(range); + } + } + + @Test + @Category({RunnableOnService.class, UsesSplittableParDo.class}) + public void testOutputAfterCheckpoint() throws Exception { + Pipeline p = TestPipeline.create(); + PCollection outputs = p.apply(Create.of("foo")) + .apply(ParDo.of(new SDFWithMultipleOutputsPerBlock())); + PAssert.thatSingleton(outputs.apply(Count.globally())) + .isEqualTo((long) SDFWithMultipleOutputsPerBlock.MAX_INDEX); + p.run(); + } + private static class SDFWithSideInputsAndOutputs extends DoFn { private final PCollectionView sideInput; private final TupleTag sideOutput; From 30ff1ee17bb290f2b50fd082d8cb63d48280c5c2 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 8 Dec 2016 15:22:35 -0800 Subject: [PATCH 132/279] Add support for having an empty CountingInput/CountingSource --- .../org/apache/beam/sdk/io/CountingInput.java | 12 ++++++---- .../apache/beam/sdk/io/CountingSource.java | 12 ++++++---- .../apache/beam/sdk/io/CountingInputTest.java | 23 ++++++++++++++++++- .../beam/sdk/io/CountingSourceTest.java | 10 ++++++++ 4 files changed, 48 insertions(+), 9 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java index 3148d8dd74d3..ac70aca3418e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java @@ -75,17 +75,21 @@ public class CountingInput { * from {@code 0} to {@code numElements - 1}. */ public static BoundedCountingInput upTo(long numElements) { - checkArgument(numElements > 0, "numElements (%s) must be greater than 0", numElements); + checkArgument(numElements >= 0, + "numElements (%s) must be greater than or equal to 0", + numElements); return new BoundedCountingInput(numElements); } /** * Creates a {@link BoundedCountingInput} that will produce elements - * starting from {@code startIndex} to {@code endIndex - 1}. + * starting from {@code startIndex} (inclusive) to {@code endIndex} (exclusive). + * If {@code startIndex == endIndex}, then no elements will be produced. */ public static BoundedCountingInput forSubrange(long startIndex, long endIndex) { - checkArgument(endIndex > startIndex, "endIndex (%s) must be greater than startIndex (%s)", - endIndex, startIndex); + checkArgument(endIndex >= startIndex, + "endIndex (%s) must be greater than or equal to startIndex (%s)", + endIndex, startIndex); return new BoundedCountingInput(startIndex, endIndex); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index bc7fb789b561..9752dba84f21 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -78,17 +78,21 @@ public class CountingSource { */ @Deprecated public static BoundedSource upTo(long numElements) { - checkArgument(numElements > 0, "numElements (%s) must be greater than 0", numElements); + checkArgument(numElements >= 0, + "numElements (%s) must be greater than or equal to 0", + numElements); return new BoundedCountingSource(0, numElements); } /** * Creates a {@link BoundedSource} that will produce elements - * from {@code startIndex} to {@code endIndex - 1}. + * starting from {@code startIndex} (inclusive) to {@code endIndex} (exclusive). + * If {@code startIndex == endIndex}, then no elements will be produced. */ static BoundedSource createSourceForSubrange(long startIndex, long endIndex) { - checkArgument(endIndex > startIndex, "endIndex (%s) must be greater than startIndex (%s)", - endIndex, startIndex); + checkArgument(endIndex >= startIndex, + "endIndex (%s) must be greater than or equal to startIndex (%s)", + endIndex, startIndex); return new BoundedCountingSource(startIndex, endIndex); } 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 02b4ba072851..4349f66ccb6f 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 @@ -71,12 +71,33 @@ public static void addCountingAsserts(PCollection input, long start, long public void testBoundedInput() { Pipeline p = TestPipeline.create(); long numElements = 1000; - PCollection input = p.apply(CountingInput.upTo(numElements)); + PCollection input = p.apply(Read.from(CountingSource.upTo(numElements))); addCountingAsserts(input, 0, numElements); p.run(); } + @Test + @Category(RunnableOnService.class) + public void testEmptyBoundedSource() { + Pipeline p = TestPipeline.create(); + PCollection input = p.apply(CountingInput.upTo(0)); + + PAssert.that(input).empty(); + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testEmptyBoundedSourceUsingRange() { + Pipeline p = TestPipeline.create(); + PCollection input = p.apply(CountingInput.forSubrange(42, 42)); + + PAssert.that(input).empty(); + p.run(); + } + + @Test @Category(RunnableOnService.class) public void testBoundedInputSubrange() { 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 88c68d3481c6..5eccde6f2282 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 @@ -90,6 +90,16 @@ public void testBoundedSource() { p.run(); } + @Test + @Category(RunnableOnService.class) + public void testEmptyBoundedSource() { + Pipeline p = TestPipeline.create(); + PCollection input = p.apply(Read.from(CountingSource.upTo(0))); + + PAssert.that(input).empty(); + p.run(); + } + @Test @Category(RunnableOnService.class) public void testBoundedSourceSplits() throws Exception { From 5169e492578a3759e20b50e36ace61bc86636ad2 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 8 Dec 2016 19:17:21 -0800 Subject: [PATCH 133/279] fixup! Fix CountingInput naming --- .../test/java/org/apache/beam/sdk/io/CountingInputTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 4349f66ccb6f..063bead8d12c 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 @@ -71,7 +71,7 @@ public static void addCountingAsserts(PCollection input, long start, long public void testBoundedInput() { Pipeline p = TestPipeline.create(); long numElements = 1000; - PCollection input = p.apply(Read.from(CountingSource.upTo(numElements))); + PCollection input = p.apply(CountingInput.upTo(numElements))); addCountingAsserts(input, 0, numElements); p.run(); @@ -79,7 +79,7 @@ public void testBoundedInput() { @Test @Category(RunnableOnService.class) - public void testEmptyBoundedSource() { + public void testEmptyBoundedInput() { Pipeline p = TestPipeline.create(); PCollection input = p.apply(CountingInput.upTo(0)); @@ -89,7 +89,7 @@ public void testEmptyBoundedSource() { @Test @Category(RunnableOnService.class) - public void testEmptyBoundedSourceUsingRange() { + public void testEmptyBoundedInputSubrange() { Pipeline p = TestPipeline.create(); PCollection input = p.apply(CountingInput.forSubrange(42, 42)); From 9bab78b55fe4661e9a221969441d8a2a4716f7e1 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 8 Dec 2016 19:20:22 -0800 Subject: [PATCH 134/279] fixup! Fix extraneous brace --- .../src/test/java/org/apache/beam/sdk/io/CountingInputTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 063bead8d12c..dfc49193fa9b 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 @@ -71,7 +71,7 @@ public static void addCountingAsserts(PCollection input, long start, long public void testBoundedInput() { Pipeline p = TestPipeline.create(); long numElements = 1000; - PCollection input = p.apply(CountingInput.upTo(numElements))); + PCollection input = p.apply(CountingInput.upTo(numElements)); addCountingAsserts(input, 0, numElements); p.run(); From 9bcba398c7516437c00517e03d75e27544b01166 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 9 Dec 2016 15:15:19 +0800 Subject: [PATCH 135/279] DataflowRunner: bump environment major version --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 00c94d0b0a80..22f6f5afbcbd 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -205,7 +205,7 @@ public class DataflowRunner extends PipelineRunner { private DataflowRunnerHooks hooks; // Environment version information. - private static final String ENVIRONMENT_MAJOR_VERSION = "5"; + private static final String ENVIRONMENT_MAJOR_VERSION = "6"; // Default Docker container images that execute Dataflow worker harness, residing in Google // Container Registry, separately for Batch and Streaming. @@ -218,7 +218,7 @@ public class DataflowRunner extends PipelineRunner { private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; @VisibleForTesting - static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1 * 1024 * 1024; + static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024; private final Set> pcollectionsRequiringIndexedFormat; From e48b0e6bc20d8eba2968decf7ac2b4ee7503a4df Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 8 Dec 2016 23:33:40 -0800 Subject: [PATCH 136/279] Remove misc uses of OldDoFn --- .../game/utils/WriteWindowedToBigQuery.java | 5 +---- .../dataflow/internal/AssignWindows.java | 8 ++++---- .../DataflowPipelineTranslatorTest.java | 17 +++++++++-------- 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java index c32289f46381..7a4fb2c8c195 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollection; @@ -43,9 +42,7 @@ public WriteWindowedToBigQuery(String tableName, } /** Convert each key/score pair into a BigQuery TableRow. */ - protected class BuildRowFn extends DoFn - implements RequiresWindowAccess { - + protected class BuildRowFn extends DoFn { @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java index 68ee7bc7e23c..27fe13d76b1d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.internal; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.Window; @@ -63,9 +63,9 @@ public PCollection expand(PCollection input) { } else { // If the windowFn didn't change, we just run a pass-through transform and then set the // new windowing strategy. - return input.apply("Identity", ParDo.of(new OldDoFn() { - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception { + return input.apply("Identity", ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { c.output(c.element()); } })).setWindowingStrategyInternal(outputStrategy); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index ac4f2dff42e9..8d0b83aa97d6 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -70,7 +70,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; @@ -508,7 +508,7 @@ private static OutputReference getOutputPortReference(Step step) throws Exceptio } /** - * Returns a Step for a OldDoFn by creating and translating a pipeline. + * Returns a Step for a {@link DoFn} by creating and translating a pipeline. */ private static Step createPredefinedStep() throws Exception { DataflowPipelineOptions options = buildPipelineOptions(); @@ -533,8 +533,9 @@ private static Step createPredefinedStep() throws Exception { return step; } - private static class NoOpFn extends OldDoFn { - @Override public void processElement(ProcessContext c) throws Exception { + private static class NoOpFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { c.output(c.element()); } } @@ -899,8 +900,8 @@ public void testStepDisplayData() throws Exception { DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); Pipeline pipeline = Pipeline.create(options); - OldDoFn fn1 = new OldDoFn() { - @Override + DoFn fn1 = new DoFn() { + @ProcessElement public void processElement(ProcessContext c) throws Exception { c.output(c.element()); } @@ -915,8 +916,8 @@ public void populateDisplayData(DisplayData.Builder builder) { } }; - OldDoFn fn2 = new OldDoFn() { - @Override + DoFn fn2 = new DoFn() { + @ProcessElement public void processElement(ProcessContext c) throws Exception { c.output(c.element()); } From f83370d69d193818e5be9ff613498f9f3296a658 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 9 Dec 2016 10:52:46 -0800 Subject: [PATCH 137/279] Enable the DirectRunner by default in Examples Archetype This ensures a runner will be on the classpath if no profile is specified. This matches the generated examples with the quickstart. --- .../examples/src/main/resources/archetype-resources/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 95d163c27cc6..9bbc6463e489 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -124,6 +124,9 @@ direct-runner + + true + From 9875b52cc3b6e1af2cbde850c6d3f5b1e75e313e Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 9 Dec 2016 09:49:49 +0800 Subject: [PATCH 138/279] [BEAM-1093] Change Dataflow to Beam in StateInternals' Javadocs --- .../java/org/apache/beam/sdk/util/state/StateInternals.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java index 24b7226738bd..0385c0b1f7ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternals.java @@ -34,7 +34,7 @@ * are persisted together with the completion status of the processing that produced these * writes. * - *

      This is a low-level API intended for use by the Dataflow SDK. It should not be + *

      This is a low-level API intended for use by the Beam SDK. It should not be * used directly, and is highly likely to change. */ @Experimental(Kind.STATE) From 1e44cb12c2663b1353717bf9237618df74684102 Mon Sep 17 00:00:00 2001 From: bchambers Date: Thu, 8 Dec 2016 10:40:17 -0800 Subject: [PATCH 139/279] Change Dataflow profiling option to saveProfilesToGcs --- .../beam/runners/dataflow/DataflowRunner.java | 5 + .../options/DataflowProfilingOptions.java | 8 +- .../DataflowPipelineTranslatorTest.java | 36 ++- .../runners/dataflow/DataflowRunnerTest.java | 263 +++++++++++------- .../options/DataflowProfilingOptionsTest.java | 6 +- 5 files changed, 189 insertions(+), 129 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 22f6f5afbcbd..85318e6ad14a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -40,6 +40,7 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; +import com.google.common.base.Strings; import com.google.common.base.Utf8; import com.google.common.collect.ForwardingMap; import com.google.common.collect.HashMultimap; @@ -262,6 +263,10 @@ public static DataflowRunner fromOptions(PipelineOptions options) { "DataflowRunner requires stagingLocation, and it is missing in PipelineOptions."); validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation()); + if (!Strings.isNullOrEmpty(dataflowOptions.getSaveProfilesToGcs())) { + validator.validateOutputFilePrefixSupported(dataflowOptions.getSaveProfilesToGcs()); + } + if (dataflowOptions.getFilesToStage() == null) { dataflowOptions.setFilesToStage(detectClassPathResourcesToStage( DataflowRunner.class.getClassLoader())); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java index 092c17a2a04f..a87d688f05ea 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java @@ -30,10 +30,10 @@ @Hidden public interface DataflowProfilingOptions { - @Description("Whether to periodically dump profiling information to local disk.\n" - + "WARNING: Enabling this option may fill local disk with profiling information.") - boolean getEnableProfilingAgent(); - void setEnableProfilingAgent(boolean enabled); + @Description("When set to a non-empty value, enables recording profiles and saving them to GCS.\n" + + "Profiles will continue until the pipeline is stopped or updated without this option.\n") + String getSaveProfilesToGcs(); + void setSaveProfilesToGcs(String gcsPath); @Description( "[INTERNAL] Additional configuration for the profiling agent. Not typically necessary.") diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 8d0b83aa97d6..ab82941f02a9 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -22,6 +22,7 @@ import static org.apache.beam.sdk.util.Structs.getString; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; @@ -49,7 +50,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -76,6 +76,7 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.GcsPathValidator; import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.Structs; @@ -188,27 +189,22 @@ public void testSettingOfSdkPipelineOptions() throws IOException { p, (DataflowRunner) p.getRunner(), Collections.emptyList()) .getJob(); - // Note that the contents of this materialized map may be changed by the act of reading an - // option, which will cause the default to get materialized whereas it would otherwise be - // left absent. It is permissible to simply alter this test to reflect current behavior. - Map settings = new HashMap<>(); - settings.put("appName", "DataflowPipelineTranslatorTest"); - settings.put("project", "some-project"); - settings.put("pathValidatorClass", - "org.apache.beam.sdk.util.GcsPathValidator"); - settings.put("runner", "org.apache.beam.runners.dataflow.DataflowRunner"); - settings.put("jobName", "some-job-name"); - settings.put("tempLocation", "gs://somebucket/some/path"); - settings.put("gcpTempLocation", "gs://somebucket/some/path"); - settings.put("stagingLocation", "gs://somebucket/some/path/staging"); - settings.put("stableUniqueNames", "WARNING"); - settings.put("streaming", false); - settings.put("numberOfWorkerHarnessThreads", 0); - settings.put("experiments", null); - Map sdkPipelineOptions = job.getEnvironment().getSdkPipelineOptions(); assertThat(sdkPipelineOptions, hasKey("options")); - assertEquals(settings, sdkPipelineOptions.get("options")); + Map optionsMap = (Map) sdkPipelineOptions.get("options"); + + assertThat(optionsMap, hasEntry("appName", (Object) "DataflowPipelineTranslatorTest")); + assertThat(optionsMap, hasEntry("project", (Object) "some-project")); + assertThat(optionsMap, + hasEntry("pathValidatorClass", (Object) GcsPathValidator.class.getName())); + assertThat(optionsMap, hasEntry("runner", (Object) DataflowRunner.class.getName())); + assertThat(optionsMap, hasEntry("jobName", (Object) "some-job-name")); + assertThat(optionsMap, hasEntry("tempLocation", (Object) "gs://somebucket/some/path")); + assertThat(optionsMap, + hasEntry("stagingLocation", (Object) "gs://somebucket/some/path/staging")); + assertThat(optionsMap, hasEntry("stableUniqueNames", (Object) "WARNING")); + assertThat(optionsMap, hasEntry("streaming", (Object) false)); + assertThat(optionsMap, hasEntry("numberOfWorkerHarnessThreads", (Object) 0)); } @Test diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 1959be55ca97..133ae8af8e0c 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -34,6 +34,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isA; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -110,6 +111,7 @@ import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Instant; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; @@ -128,6 +130,11 @@ @RunWith(JUnit4.class) public class DataflowRunnerTest { + private static final String VALID_STAGING_BUCKET = "gs://valid-bucket/staging"; + private static final String VALID_TEMP_BUCKET = "gs://valid-bucket/temp"; + private static final String VALID_PROFILE_BUCKET = "gs://valid-bucket/profiles"; + private static final String NON_EXISTENT_BUCKET = "gs://non-existent-bucket/location"; + private static final String PROJECT_ID = "some-project"; @Rule @@ -137,6 +144,9 @@ public class DataflowRunnerTest { @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(DataflowRunner.class); + private Dataflow.Projects.Jobs mockJobs; + private GcsUtil mockGcsUtil; + // Asserts that the given Job has all expected fields set. private static void assertValidJob(Job job) { assertNull(job.getId()); @@ -144,6 +154,38 @@ private static void assertValidJob(Job job) { assertTrue(Pattern.matches("[a-z]([-a-z0-9]*[a-z0-9])?", job.getName())); } + @Before + public void setUp() throws IOException { + this.mockGcsUtil = mock(GcsUtil.class); + when(mockGcsUtil.create(any(GcsPath.class), anyString())) + .then(new Answer() { + @Override + public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable { + return FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE); + } + }); + when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true); + when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer>() { + @Override + public List answer(InvocationOnMock invocation) throws Throwable { + return ImmutableList.of((GcsPath) invocation.getArguments()[0]); + } + }); + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_STAGING_BUCKET))).thenReturn(true); + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_TEMP_BUCKET))).thenReturn(true); + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_TEMP_BUCKET + "/staging"))). + thenReturn(true); + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_PROFILE_BUCKET))).thenReturn(true); + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(NON_EXISTENT_BUCKET))).thenReturn(false); + + // The dataflow pipeline attempts to output to this location. + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri("gs://bucket/object"))).thenReturn(true); + + mockJobs = mock(Dataflow.Projects.Jobs.class); + } + private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) { options.setStableUniqueNames(CheckEnabled.ERROR); options.setRunner(DataflowRunner.class); @@ -155,19 +197,16 @@ private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) { return p; } - private static Dataflow buildMockDataflow( - final ArgumentCaptor jobCaptor) throws IOException { + private Dataflow buildMockDataflow() throws IOException { Dataflow mockDataflowClient = mock(Dataflow.class); Dataflow.Projects mockProjects = mock(Dataflow.Projects.class); - Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class); Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class); Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class); when(mockDataflowClient.projects()).thenReturn(mockProjects); when(mockProjects.jobs()).thenReturn(mockJobs); - when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture())) - .thenReturn(mockRequest); + when(mockJobs.create(eq(PROJECT_ID), isA(Job.class))).thenReturn(mockRequest); when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList); when(mockList.setPageToken(anyString())).thenReturn(mockList); when(mockList.execute()) @@ -186,25 +225,17 @@ private static Dataflow buildMockDataflow( return mockDataflowClient; } - /** - * Build a mock {@link GcsUtil} with return values. - * - * @param bucketExist first return value - * @param bucketAccessible next return values - */ - private GcsUtil buildMockGcsUtil(Boolean bucketExist, Boolean... bucketAccessible) - throws IOException { + private GcsUtil buildMockGcsUtil() throws IOException { GcsUtil mockGcsUtil = mock(GcsUtil.class); when(mockGcsUtil.create(any(GcsPath.class), anyString())) .then(new Answer() { - @Override - public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable { - return FileChannel.open( - Files.createTempFile("channel-", ".tmp"), - StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE); - } - }); - + @Override + public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable { + return FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE); + } + }); when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true); when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer>() { @Override @@ -212,26 +243,18 @@ public List answer(InvocationOnMock invocation) throws Throwable { return ImmutableList.of((GcsPath) invocation.getArguments()[0]); } }); - when(mockGcsUtil.bucketAccessible(any(GcsPath.class))) - .thenReturn(bucketExist, bucketAccessible); return mockGcsUtil; } private DataflowPipelineOptions buildPipelineOptions() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - return buildPipelineOptions(jobCaptor); - } - - private DataflowPipelineOptions buildPipelineOptions( - ArgumentCaptor jobCaptor) throws IOException { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setRunner(DataflowRunner.class); options.setProject(PROJECT_ID); - options.setTempLocation("gs://somebucket/some/path"); + options.setTempLocation(VALID_TEMP_BUCKET); // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath. options.setFilesToStage(new LinkedList()); - options.setDataflowClient(buildMockDataflow(jobCaptor)); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setDataflowClient(buildMockDataflow()); + options.setGcsUtil(mockGcsUtil); options.setGcpCredential(new TestCredential()); return options; } @@ -271,22 +294,22 @@ public void testPathValidatorOverride() { @Test public void testFromOptionsWithUppercaseConvertsToLowercase() throws Exception { String mixedCase = "ThisJobNameHasMixedCase"; - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipelineOptions options = buildPipelineOptions(); options.setJobName(mixedCase); - DataflowRunner runner = DataflowRunner.fromOptions(options); + DataflowRunner.fromOptions(options); assertThat(options.getJobName(), equalTo(mixedCase.toLowerCase())); } @Test public void testRun() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipelineOptions options = buildPipelineOptions(); Pipeline p = buildDataflowPipeline(options); DataflowPipelineJob job = (DataflowPipelineJob) p.run(); assertEquals("newid", job.getJobId()); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } @@ -317,14 +340,15 @@ public void testRunReturnDifferentRequestId() throws IOException { @Test public void testUpdate() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipelineOptions options = buildPipelineOptions(); options.setUpdate(true); options.setJobName("oldJobName"); Pipeline p = buildDataflowPipeline(options); DataflowPipelineJob job = (DataflowPipelineJob) p.run(); assertEquals("newid", job.getJobId()); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } @@ -378,9 +402,6 @@ protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) { public void testRunWithFiles() throws IOException { // Test that the function DataflowRunner.stageFiles works as // expected. - GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */); - final String gcsStaging = "gs://somebucket/some/path"; - final String gcsTemp = "gs://somebucket/some/temp/path"; final String cloudDataflowDataset = "somedataset"; // Create some temporary files. @@ -391,17 +412,16 @@ public void testRunWithFiles() throws IOException { String overridePackageName = "alias.txt"; - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setFilesToStage(ImmutableList.of( temp1.getAbsolutePath(), overridePackageName + "=" + temp2.getAbsolutePath())); - options.setStagingLocation(gcsStaging); - options.setTempLocation(gcsTemp); + options.setStagingLocation(VALID_STAGING_BUCKET); + options.setTempLocation(VALID_TEMP_BUCKET); options.setTempDatasetId(cloudDataflowDataset); options.setProject(PROJECT_ID); options.setJobName("job"); - options.setDataflowClient(buildMockDataflow(jobCaptor)); + options.setDataflowClient(buildMockDataflow()); options.setGcsUtil(mockGcsUtil); options.setGcpCredential(new TestCredential()); @@ -410,6 +430,8 @@ public void testRunWithFiles() throws IOException { DataflowPipelineJob job = (DataflowPipelineJob) p.run(); assertEquals("newid", job.getJobId()); + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); Job workflowJob = jobCaptor.getValue(); assertValidJob(workflowJob); @@ -424,7 +446,7 @@ public void testRunWithFiles() throws IOException { assertEquals(overridePackageName, workflowPackage2.getName()); assertEquals( - "storage.googleapis.com/somebucket/some/temp/path", + GcsPath.fromUri(VALID_TEMP_BUCKET).toResourceName(), workflowJob.getEnvironment().getTempStoragePrefix()); assertEquals( cloudDataflowDataset, @@ -481,15 +503,12 @@ public void detectClassPathResourceWithNonFileResources() throws Exception { @Test public void testGcsStagingLocationInitialization() throws Exception { - // Test that the staging location is initialized correctly. - String gcsTemp = "gs://somebucket/some/temp/path"; - // Set temp location (required), and check that staging location is set. DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); - options.setTempLocation(gcsTemp); + options.setTempLocation(VALID_TEMP_BUCKET); options.setProject(PROJECT_ID); options.setGcpCredential(new TestCredential()); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcsUtil(mockGcsUtil); options.setRunner(DataflowRunner.class); DataflowRunner.fromOptions(options); @@ -499,9 +518,7 @@ public void testGcsStagingLocationInitialization() throws Exception { @Test public void testNonGcsFilePathInReadFailure() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor)); + Pipeline p = buildDataflowPipeline(buildPipelineOptions()); p.apply("ReadMyNonGcsFile", TextIO.Read.from(tmpFolder.newFile().getPath())); thrown.expectCause(Matchers.allOf( @@ -509,12 +526,16 @@ public void testNonGcsFilePathInReadFailure() throws IOException { ThrowableMessageMatcher.hasMessage( containsString("expected a valid 'gs://' path but was given")))); p.run(); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } @Test public void testNonGcsFilePathInWriteFailure() throws IOException { Pipeline p = buildDataflowPipeline(buildPipelineOptions()); + PCollection pc = p.apply("ReadMyGcsFile", TextIO.Read.from("gs://bucket/object")); thrown.expect(IllegalArgumentException.class); @@ -524,15 +545,16 @@ public void testNonGcsFilePathInWriteFailure() throws IOException { @Test public void testMultiSlashGcsFileReadPath() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor)); + Pipeline p = buildDataflowPipeline(buildPipelineOptions()); p.apply("ReadInvalidGcsFile", TextIO.Read.from("gs://bucket/tmp//file")); thrown.expectCause(Matchers.allOf( instanceOf(IllegalArgumentException.class), ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes")))); p.run(); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } @@ -548,22 +570,21 @@ public void testMultiSlashGcsFileWritePath() throws IOException { @Test public void testInvalidGcpTempLocation() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipelineOptions options = buildPipelineOptions(); options.setGcpTempLocation("file://temp/location"); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(containsString("expected a valid 'gs://' path but was given")); DataflowRunner.fromOptions(options); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } @Test public void testNonGcsTempLocation() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipelineOptions options = buildPipelineOptions(); options.setTempLocation("file://temp/location"); thrown.expect(IllegalArgumentException.class); @@ -592,39 +613,68 @@ public void testInvalidStagingLocation() throws IOException { } @Test - public void testNonExistentTempLocation() throws IOException { - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + public void testInvalidProfileLocation() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setSaveProfilesToGcs("file://my/staging/location"); + try { + DataflowRunner.fromOptions(options); + fail("fromOptions should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given")); + } + options.setSaveProfilesToGcs("my/staging/location"); + try { + DataflowRunner.fromOptions(options); + fail("fromOptions should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given")); + } + } - GcsUtil mockGcsUtil = - buildMockGcsUtil(false /* temp bucket exists */, true /* staging bucket exists */); - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); - options.setGcsUtil(mockGcsUtil); - options.setGcpTempLocation("gs://non-existent-bucket/location"); + @Test + public void testNonExistentTempLocation() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setGcpTempLocation(NON_EXISTENT_BUCKET); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(containsString( - "Output path does not exist or is not writeable: gs://non-existent-bucket/location")); + "Output path does not exist or is not writeable: " + NON_EXISTENT_BUCKET)); DataflowRunner.fromOptions(options); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } @Test public void testNonExistentStagingLocation() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setStagingLocation(NON_EXISTENT_BUCKET); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString( + "Output path does not exist or is not writeable: " + NON_EXISTENT_BUCKET)); + DataflowRunner.fromOptions(options); + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); + assertValidJob(jobCaptor.getValue()); + } - GcsUtil mockGcsUtil = - buildMockGcsUtil(true /* temp bucket exists */, false /* staging bucket exists */); - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); - options.setGcpTempLocation(options.getTempLocation()); // bypass validation for GcpTempLocation - options.setGcsUtil(mockGcsUtil); - options.setStagingLocation("gs://non-existent-bucket/location"); + @Test + public void testNonExistentProfileLocation() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setSaveProfilesToGcs(NON_EXISTENT_BUCKET); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(containsString( - "Output path does not exist or is not writeable: gs://non-existent-bucket/location")); + "Output path does not exist or is not writeable: " + NON_EXISTENT_BUCKET)); DataflowRunner.fromOptions(options); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); - } + } @Test public void testNoProjectFails() { @@ -648,8 +698,8 @@ public void testProjectId() throws IOException { options.setRunner(DataflowRunner.class); options.setProject("foo-12345"); - options.setGcpTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcpTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); options.setGcpCredential(new TestCredential()); DataflowRunner.fromOptions(options); @@ -661,8 +711,8 @@ public void testProjectPrefix() throws IOException { options.setRunner(DataflowRunner.class); options.setProject("google.com:some-project-12345"); - options.setGcpTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcpTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); options.setGcpCredential(new TestCredential()); DataflowRunner.fromOptions(options); @@ -674,8 +724,8 @@ public void testProjectNumber() throws IOException { options.setRunner(DataflowRunner.class); options.setProject("12345"); - options.setGcpTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcpTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Project ID"); @@ -690,8 +740,8 @@ public void testProjectDescription() throws IOException { options.setRunner(DataflowRunner.class); options.setProject("some project"); - options.setGcpTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcpTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Project ID"); @@ -706,8 +756,8 @@ public void testInvalidNumberOfWorkerHarnessThreads() throws IOException { options.setRunner(DataflowRunner.class); options.setProject("foo-12345"); - options.setTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcpTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1); @@ -731,25 +781,34 @@ public void testNoStagingLocationAndNoTempLocationFails() { } @Test - public void testStagingLocationAndNoTempLocationSucceeds() throws Exception { + public void testGcpTempAndNoTempLocationSucceeds() throws Exception { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setRunner(DataflowRunner.class); options.setGcpCredential(new TestCredential()); options.setProject("foo-project"); - options.setGcpTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setGcpTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); DataflowRunner.fromOptions(options); } @Test - public void testTempLocationAndNoStagingLocationSucceeds() throws Exception { + public void testTempLocationAndNoGcpTempLocationSucceeds() throws Exception { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setRunner(DataflowRunner.class); options.setGcpCredential(new TestCredential()); options.setProject("foo-project"); - options.setTempLocation("gs://spam/ham/eggs"); - options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */)); + options.setTempLocation(VALID_TEMP_BUCKET); + options.setGcsUtil(mockGcsUtil); + + DataflowRunner.fromOptions(options); + } + + + @Test + public void testValidProfileLocation() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setSaveProfilesToGcs(VALID_PROFILE_BUCKET); DataflowRunner.fromOptions(options); } @@ -855,10 +914,7 @@ protected Coder getDefaultOutputCoder(PCollection input) { @Test public void testTransformTranslatorMissing() throws IOException { - // Test that we throw if we don't provide a translation. - ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); - - DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipelineOptions options = buildPipelineOptions(); Pipeline p = Pipeline.create(options); p.apply(Create.of(Arrays.asList(1, 2, 3))) @@ -869,6 +925,9 @@ public void testTransformTranslatorMissing() throws IOException { DataflowPipelineTranslator.fromOptions(options) .translate( p, (DataflowRunner) p.getRunner(), Collections.emptyList()); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + Mockito.verify(mockJobs).create(eq(PROJECT_ID), jobCaptor.capture()); assertValidJob(jobCaptor.getValue()); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java index 87c74a4e37b9..299f3c85652d 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.dataflow.options; +import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -38,9 +38,9 @@ public class DataflowProfilingOptionsTest { @Test public void testOptionsObject() throws Exception { DataflowPipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] { - "--enableProfilingAgent", "--profilingAgentConfiguration={\"interval\": 21}"}) + "--saveProfilesToGcs=path", "--profilingAgentConfiguration={\"interval\": 21}"}) .as(DataflowPipelineOptions.class); - assertTrue(options.getEnableProfilingAgent()); + assertThat(options.getSaveProfilesToGcs(), equalTo("path")); String json = MAPPER.writeValueAsString(options); assertThat(json, Matchers.containsString( From 65dd62cd9b1d6f7925ee60b73a5a3fac1e4b1bf1 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 9 Dec 2016 11:36:22 -0800 Subject: [PATCH 140/279] Copy Runner Profiles to Java8 Archetype POM This enables Java8 examples to be run cross-runner --- .../resources/archetype-resources/pom.xml | 116 +++++++++++++++--- 1 file changed, 101 insertions(+), 15 deletions(-) diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index ece84126fcf7..693850ae34fb 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -26,6 +26,10 @@ jar + + 0.4.0-incubating-SNAPSHOT + + @@ -71,6 +75,103 @@ + + + direct-runner + + true + + + + + org.apache.beam + beam-runners-direct-java + ${beam.version} + runtime + + + + + + apex-runner + + + + org.apache.beam + beam-runners-apex + ${beam.version} + runtime + + + + + + dataflow-runner + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${beam.version} + runtime + + + + + + flink-runner + + + + org.apache.beam + beam-runners-flink_2.10 + ${beam.version} + runtime + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + + + spark-runner + + + + org.apache.beam + beam-runners-spark + ${beam.version} + runtime + + + org.apache.spark + spark-streaming_2.10 + 1.6.2 + runtime + + + org.slf4j + jul-to-slf4j + + + + + com.fasterxml.jackson.module + jackson-module-scala_2.10 + 2.7.2 + runtime + + + + + @@ -79,21 +180,6 @@ 0.4.0-incubating-SNAPSHOT - - - org.apache.beam - beam-runners-direct-java - 0.4.0-incubating-SNAPSHOT - runtime - - - - org.apache.beam - beam-runners-google-cloud-dataflow-java - 0.4.0-incubating-SNAPSHOT - runtime - - org.apache.beam From d7dbf16e9886e29224361781f7beb5c2087ff6d0 Mon Sep 17 00:00:00 2001 From: Sam Whittle Date: Fri, 9 Dec 2016 12:43:04 -0800 Subject: [PATCH 141/279] Fix LateDataDroppingDoFnRunner to not increment the counter on reiteration. --- .../core/LateDataDroppingDoFnRunner.java | 25 +++++++++++++------ .../core/LateDataDroppingDoFnRunnerTest.java | 3 +++ 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index b6f700f7a6e9..9bfe9aeed559 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -116,21 +116,30 @@ public WindowedValue apply(BoundedWindow window) { } }); }}); + Iterable> concatElements = Iterables.concat(windowsExpandedElements); + + // Bump the counter separately since we don't want multiple iterations to + // increase it multiple times. + for (WindowedValue input : concatElements) { + BoundedWindow window = Iterables.getOnlyElement(input.getWindows()); + if (canDropDueToExpiredWindow(window)) { + // The element is too late for this window. + droppedDueToLateness.addValue(1L); + WindowTracing.debug( + "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} " + + "since too far behind inputWatermark:{}; outputWatermark:{}", + input.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + } + } Iterable> nonLateElements = Iterables.filter( - Iterables.concat(windowsExpandedElements), + concatElements, new Predicate>() { @Override public boolean apply(WindowedValue input) { BoundedWindow window = Iterables.getOnlyElement(input.getWindows()); if (canDropDueToExpiredWindow(window)) { - // The element is too late for this window. - droppedDueToLateness.addValue(1L); - WindowTracing.debug( - "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} " - + "since too far behind inputWatermark:{}; outputWatermark:{}", - input.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); return false; } else { return true; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index 1cf05b670729..3cd5d4abaddf 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java @@ -80,6 +80,9 @@ public void testLateDataFilter() throws Exception { createDatum(18, 18L)); assertThat(expected, containsInAnyOrder(Iterables.toArray(actual, WindowedValue.class))); assertEquals(1, droppedDueToLateness.sum); + // Ensure that reiterating returns the same results and doesn't increment the counter again. + assertThat(expected, containsInAnyOrder(Iterables.toArray(actual, WindowedValue.class))); + assertEquals(1, droppedDueToLateness.sum); } private WindowedValue createDatum(T element, long timestampMillis) { From 0b38c4bd75e96f53d9955a7508e2843c22fd0fbf Mon Sep 17 00:00:00 2001 From: jasonkuster Date: Fri, 9 Dec 2016 18:41:39 -0800 Subject: [PATCH 142/279] Update badge for new suite name. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 617500636598..d919e5bd14a2 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,7 @@ fully endorsed by the ASF. _**The Apache Beam project is in the process of bootstrapping. This includes the creation of project resources, the refactoring of the initial code submissions, and the formulation of project documentation, planning, and design documents. Please expect a significant amount of churn and breaking changes in the near future.**_ [![Build Status](https://api.travis-ci.org/apache/incubator-beam.svg?branch=master)](https://travis-ci.org/apache/incubator-beam?branch=master) -[![Build Status](https://builds.apache.org/buildStatus/icon?job=beam_PostCommit_MavenVerify)](https://builds.apache.org/job/beam_PostCommit_MavenVerify/) +[![Build Status](https://builds.apache.org/buildStatus/icon?job=beam_PostCommit_Java_MavenInstall)](https://builds.apache.org/job/beam_PostCommit_MavenVerify/) [![Coverage Status](https://coveralls.io/repos/github/apache/incubator-beam/badge.svg?branch=master)](https://coveralls.io/github/apache/incubator-beam?branch=master) ## Overview From b6b2e202ae8c5d3d1c081a1e24033380d7f55593 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Thu, 24 Nov 2016 18:36:11 -0800 Subject: [PATCH 143/279] BEAM-830 Support launch on YARN cluster. --- runners/apex/pom.xml | 52 ++- .../apache/beam/runners/apex/ApexRunner.java | 48 ++- .../beam/runners/apex/ApexRunnerResult.java | 50 +-- .../beam/runners/apex/ApexYarnLauncher.java | 395 ++++++++++++++++++ .../runners/apex/ApexYarnLauncherTest.java | 138 ++++++ 5 files changed, 631 insertions(+), 52 deletions(-) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index b60423757e16..9f1455a0adb0 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -35,7 +35,7 @@ jar - 3.5.0-SNAPSHOT + 3.5.0 3.4.0 true @@ -218,22 +218,64 @@ - org.apache.apex:apex-api:jar:3.5.0-SNAPSHOT + org.apache.apex:apex-api:jar:3.5.0 org.apache.commons:commons-lang3::3.1 - commons-io:commons-io:jar: + commons-io:commons-io:jar:2.4 com.esotericsoftware.kryo:kryo::2.24.0 - com.datatorrent:netlet:: + com.datatorrent:netlet::1.3.0 org.slf4j:slf4j-api:jar:1.7.14 - org.apache.hadoop:hadoop-common:jar: + org.apache.hadoop:hadoop-common:jar:2.6.0 joda-time:joda-time:jar:2.4 com.google.guava:guava:jar:19.0 + + + dependency-tree + generate-test-resources + + tree + + + ${project.build.directory}/classes/org/apache/beam/runners/apex/dependency-tree + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [2.10,) + + tree + + + + + + + + + + + + diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java index 9507fb985a17..899efa3cee37 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -17,19 +17,22 @@ */ package org.apache.beam.runners.apex; -import static com.google.common.base.Preconditions.checkArgument; - +import com.datatorrent.api.Attribute; import com.datatorrent.api.Context.DAGContext; import com.datatorrent.api.DAG; -import com.datatorrent.api.LocalMode; import com.datatorrent.api.StreamingApplication; import com.google.common.base.Throwables; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicReference; +import org.apache.apex.api.EmbeddedAppLauncher; +import org.apache.apex.api.Launcher; +import org.apache.apex.api.Launcher.AppHandle; +import org.apache.apex.api.Launcher.LaunchMode; import org.apache.beam.runners.apex.translation.ApexPipelineTranslator; import org.apache.beam.runners.core.AssignWindows; import org.apache.beam.sdk.Pipeline; @@ -122,33 +125,44 @@ public OutputT apply( public ApexRunnerResult run(final Pipeline pipeline) { final ApexPipelineTranslator translator = new ApexPipelineTranslator(options); + final AtomicReference apexDAG = new AtomicReference<>(); StreamingApplication apexApp = new StreamingApplication() { @Override public void populateDAG(DAG dag, Configuration conf) { + apexDAG.set(dag); dag.setAttribute(DAGContext.APPLICATION_NAME, options.getApplicationName()); translator.translate(pipeline, dag); } }; - checkArgument(options.isEmbeddedExecution(), - "only embedded execution is supported at this time"); - LocalMode lma = LocalMode.newInstance(); - Configuration conf = new Configuration(false); - try { - lma.prepareDAG(apexApp, conf); - LocalMode.Controller lc = lma.getController(); + if (options.isEmbeddedExecution()) { + Launcher launcher = Launcher.getLauncher(LaunchMode.EMBEDDED); + Attribute.AttributeMap launchAttributes = new Attribute.AttributeMap.DefaultAttributeMap(); + launchAttributes.put(EmbeddedAppLauncher.RUN_ASYNC, true); if (options.isEmbeddedExecutionDebugMode()) { // turns off timeout checking for operator progress - lc.setHeartbeatMonitoringEnabled(false); + launchAttributes.put(EmbeddedAppLauncher.HEARTBEAT_MONITORING, false); + } + Configuration conf = new Configuration(false); + try { + ApexRunner.ASSERTION_ERROR.set(null); + AppHandle apexAppResult = launcher.launchApp(apexApp, conf, launchAttributes); + return new ApexRunnerResult(apexDAG.get(), apexAppResult); + } catch (Exception e) { + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); + } + } else { + try { + ApexYarnLauncher yarnLauncher = new ApexYarnLauncher(); + AppHandle apexAppResult = yarnLauncher.launchApp(apexApp); + return new ApexRunnerResult(apexDAG.get(), apexAppResult); + } catch (IOException e) { + throw new RuntimeException("Failed to launch the application on YARN.", e); } - ApexRunner.ASSERTION_ERROR.set(null); - lc.runAsync(); - return new ApexRunnerResult(lma.getDAG(), lc); - } catch (Exception e) { - Throwables.propagateIfPossible(e); - throw new RuntimeException(e); } + } /** diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java index 18b50bc935b4..85481945b6e5 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.apex; import com.datatorrent.api.DAG; -import com.datatorrent.api.LocalMode; import java.io.IOException; -import java.lang.reflect.Field; +import org.apache.apex.api.Launcher.AppHandle; +import org.apache.apex.api.Launcher.ShutdownMode; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.Pipeline; @@ -36,12 +36,12 @@ */ public class ApexRunnerResult implements PipelineResult { private final DAG apexDAG; - private final LocalMode.Controller ctrl; + private final AppHandle apexApp; private State state = State.UNKNOWN; - public ApexRunnerResult(DAG dag, LocalMode.Controller ctrl) { + public ApexRunnerResult(DAG dag, AppHandle apexApp) { this.apexDAG = dag; - this.ctrl = ctrl; + this.apexApp = apexApp; } @Override @@ -57,19 +57,31 @@ public AggregatorValues getAggregatorValues(Aggregator aggregator) @Override public State cancel() throws IOException { - ctrl.shutdown(); + apexApp.shutdown(ShutdownMode.KILL); state = State.CANCELLED; return state; } @Override public State waitUntilFinish(Duration duration) { - return ApexRunnerResult.waitUntilFinished(ctrl, duration); + long timeout = (duration == null || duration.getMillis() < 1) ? Long.MAX_VALUE + : System.currentTimeMillis() + duration.getMillis(); + try { + while (!apexApp.isFinished() && System.currentTimeMillis() < timeout) { + if (ApexRunner.ASSERTION_ERROR.get() != null) { + throw ApexRunner.ASSERTION_ERROR.get(); + } + Thread.sleep(500); + } + return apexApp.isFinished() ? State.DONE : null; + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } @Override public State waitUntilFinish() { - return ApexRunnerResult.waitUntilFinished(ctrl, null); + return waitUntilFinish(null); } @Override @@ -85,26 +97,4 @@ public DAG getApexDAG() { return apexDAG; } - public static State waitUntilFinished(LocalMode.Controller ctrl, Duration duration) { - // we need to rely on internal field for now - // Apex should make it available through API in upcoming release. - long timeout = (duration == null || duration.getMillis() < 1) ? Long.MAX_VALUE - : System.currentTimeMillis() + duration.getMillis(); - Field appDoneField; - try { - appDoneField = ctrl.getClass().getDeclaredField("appDone"); - appDoneField.setAccessible(true); - while (!appDoneField.getBoolean(ctrl) && System.currentTimeMillis() < timeout) { - if (ApexRunner.ASSERTION_ERROR.get() != null) { - throw ApexRunner.ASSERTION_ERROR.get(); - } - Thread.sleep(500); - } - return appDoneField.getBoolean(ctrl) ? State.DONE : null; - } catch (NoSuchFieldException | SecurityException | IllegalArgumentException - | IllegalAccessException | InterruptedException e) { - throw new RuntimeException(e); - } - } - } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java new file mode 100644 index 000000000000..0ae4cc7dee90 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java @@ -0,0 +1,395 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.apex; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.datatorrent.api.Attribute; +import com.datatorrent.api.Attribute.AttributeMap; +import com.datatorrent.api.DAG; +import com.datatorrent.api.StreamingApplication; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Sets; + +import java.io.BufferedReader; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.Serializable; +import java.lang.reflect.AccessibleObject; +import java.lang.reflect.Field; +import java.net.URI; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.file.FileSystem; +import java.nio.file.FileSystems; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.jar.JarFile; +import java.util.jar.Manifest; + +import org.apache.apex.api.EmbeddedAppLauncher; +import org.apache.apex.api.Launcher; +import org.apache.apex.api.Launcher.AppHandle; +import org.apache.apex.api.Launcher.LaunchMode; +import org.apache.apex.api.Launcher.LauncherException; +import org.apache.apex.api.Launcher.ShutdownMode; +import org.apache.apex.api.YarnAppLauncher; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.SerializationUtils; +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Proxy to launch the YARN application through the hadoop script to run in the + * pre-configured environment (class path, configuration, native libraries etc.). + * + *

      The proxy takes the DAG and communicates with the Hadoop services to launch + * it on the cluster. + */ +public class ApexYarnLauncher { + private static final Logger LOG = LoggerFactory.getLogger(ApexYarnLauncher.class); + + public AppHandle launchApp(StreamingApplication app) throws IOException { + + List jarsToShip = getYarnDeployDependencies(); + StringBuilder classpath = new StringBuilder(); + for (File path : jarsToShip) { + if (path.isDirectory()) { + File tmpJar = File.createTempFile("beam-runners-apex-", ".jar"); + createJar(path, tmpJar); + tmpJar.deleteOnExit(); + path = tmpJar; + } + if (classpath.length() != 0) { + classpath.append(':'); + } + classpath.append(path.getAbsolutePath()); + } + + EmbeddedAppLauncher embeddedLauncher = Launcher.getLauncher(LaunchMode.EMBEDDED); + DAG dag = embeddedLauncher.getDAG(); + app.populateDAG(dag, new Configuration(false)); + + Attribute.AttributeMap launchAttributes = new Attribute.AttributeMap.DefaultAttributeMap(); + launchAttributes.put(YarnAppLauncher.LIB_JARS, classpath.toString().replace(':', ',')); + LaunchParams lp = new LaunchParams(dag, launchAttributes); + lp.cmd = "hadoop " + ApexYarnLauncher.class.getName(); + HashMap env = new HashMap<>(); + env.put("HADOOP_USER_CLASSPATH_FIRST", "1"); + env.put("HADOOP_CLASSPATH", classpath.toString()); + lp.env = env; + return launchApp(lp); + } + + protected AppHandle launchApp(LaunchParams params) throws IOException { + File tmpFile = File.createTempFile("beam-runner-apex", "params"); + tmpFile.deleteOnExit(); + try (FileOutputStream fos = new FileOutputStream(tmpFile)) { + SerializationUtils.serialize(params, fos); + } + if (params.getCmd() == null) { + ApexYarnLauncher.main(new String[] {tmpFile.getAbsolutePath()}); + } else { + String cmd = params.getCmd() + " " + tmpFile.getAbsolutePath(); + ByteArrayOutputStream consoleOutput = new ByteArrayOutputStream(); + LOG.info("Executing: {} with {}", cmd, params.getEnv()); + + ProcessBuilder pb = new ProcessBuilder("bash", "-c", cmd); + Map env = pb.environment(); + env.putAll(params.getEnv()); + Process p = pb.start(); + ProcessWatcher pw = new ProcessWatcher(p); + InputStream output = p.getInputStream(); + InputStream error = p.getErrorStream(); + while (!pw.isFinished()) { + IOUtils.copy(output, consoleOutput); + IOUtils.copy(error, consoleOutput); + } + if (pw.rc != 0) { + String msg = "The Beam Apex runner in non-embedded mode requires the Hadoop client" + + " to be installed on the machine from which you launch the job" + + " and the 'hadoop' script in $PATH"; + LOG.error(msg); + throw new RuntimeException("Failed to run: " + cmd + " (exit code " + pw.rc + ")" + "\n" + + consoleOutput.toString()); + } + } + return new AppHandle() { + @Override + public boolean isFinished() { + // TODO (future PR): interaction with child process + LOG.warn("YARN application runs asynchronously and status check not implemented."); + return true; + } + @Override + public void shutdown(ShutdownMode arg0) throws LauncherException { + // TODO (future PR): interaction with child process + throw new UnsupportedOperationException(); + } + }; + } + + /** + * From the current classpath, find the jar files that need to be deployed + * with the application to run on YARN. Hadoop dependencies are provided + * through the Hadoop installation and the application should not bundle them + * to avoid conflicts. This is done by removing the Hadoop compile + * dependencies (transitively) by parsing the Maven dependency tree. + * + * @return list of jar files to ship + * @throws IOException when dependency information cannot be read + */ + public static List getYarnDeployDependencies() throws IOException { + InputStream dependencyTree = ApexRunner.class.getResourceAsStream("dependency-tree"); + BufferedReader br = new BufferedReader(new InputStreamReader(dependencyTree)); + String line = null; + List excludes = new ArrayList<>(); + int excludeLevel = Integer.MAX_VALUE; + while ((line = br.readLine()) != null) { + for (int i = 0; i < line.length(); i++) { + char c = line.charAt(i); + if (Character.isLetter(c)) { + if (i > excludeLevel) { + excludes.add(line.substring(i)); + } else { + if (line.substring(i).startsWith("org.apache.hadoop")) { + excludeLevel = i; + excludes.add(line.substring(i)); + } else { + excludeLevel = Integer.MAX_VALUE; + } + } + break; + } + } + } + br.close(); + + Set excludeJarFileNames = Sets.newHashSet(); + for (String exclude : excludes) { + String[] mvnc = exclude.split(":"); + String fileName = mvnc[1] + "-"; + if (mvnc.length == 6) { + fileName += mvnc[4] + "-" + mvnc[3]; // with classifier + } else { + fileName += mvnc[3]; + } + fileName += ".jar"; + excludeJarFileNames.add(fileName); + } + + ClassLoader classLoader = ApexYarnLauncher.class.getClassLoader(); + URL[] urls = ((URLClassLoader) classLoader).getURLs(); + List dependencyJars = new ArrayList<>(); + for (int i = 0; i < urls.length; i++) { + File f = new File(urls[i].getFile()); + // dependencies can also be directories in the build reactor, + // the Apex client will automatically create jar files for those. + if (f.exists() && !excludeJarFileNames.contains(f.getName())) { + dependencyJars.add(f); + } + } + return dependencyJars; + } + + /** + * Create a jar file from the given directory. + * @param dir source directory + * @param jarFile jar file name + * @throws IOException when file cannot be created + */ + public static void createJar(File dir, File jarFile) throws IOException { + + final Map env = Collections.singletonMap("create", "true"); + if (jarFile.exists() && !jarFile.delete()) { + throw new RuntimeException("Failed to remove " + jarFile); + } + URI uri = URI.create("jar:" + jarFile.toURI()); + try (final FileSystem zipfs = FileSystems.newFileSystem(uri, env);) { + + File manifestFile = new File(dir, JarFile.MANIFEST_NAME); + Files.createDirectory(zipfs.getPath("META-INF")); + final OutputStream out = Files.newOutputStream(zipfs.getPath(JarFile.MANIFEST_NAME)); + if (!manifestFile.exists()) { + new Manifest().write(out); + } else { + FileUtils.copyFile(manifestFile, out); + } + out.close(); + + final java.nio.file.Path root = dir.toPath(); + Files.walkFileTree(root, new java.nio.file.SimpleFileVisitor() { + String relativePath; + + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) + throws IOException { + relativePath = root.relativize(dir).toString(); + if (!relativePath.isEmpty()) { + if (!relativePath.endsWith("/")) { + relativePath += "/"; + } + final Path dstDir = zipfs.getPath(relativePath); + Files.createDirectory(dstDir); + } + return super.preVisitDirectory(dir, attrs); + } + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + String name = relativePath + file.getFileName(); + if (!JarFile.MANIFEST_NAME.equals(name)) { + final OutputStream out = Files.newOutputStream(zipfs.getPath(name)); + FileUtils.copyFile(file.toFile(), out); + out.close(); + } + return super.visitFile(file, attrs); + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + relativePath = root.relativize(dir.getParent()).toString(); + if (!relativePath.isEmpty() && !relativePath.endsWith("/")) { + relativePath += "/"; + } + return super.postVisitDirectory(dir, exc); + } + }); + } + } + + /** + * The main method expects the serialized DAG and will launch the YARN application. + * @param args location of launch parameters + * @throws IOException when parameters cannot be read + */ + public static void main(String[] args) throws IOException { + checkArgument(args.length == 1, "exactly one argument expected"); + File file = new File(args[0]); + checkArgument(file.exists() && file.isFile(), "invalid file path %s", file); + final LaunchParams params = (LaunchParams) SerializationUtils.deserialize( + new FileInputStream(file)); + StreamingApplication apexApp = new StreamingApplication() { + @Override + public void populateDAG(DAG dag, Configuration conf) { + copyShallow(params.dag, dag); + } + }; + Configuration conf = new Configuration(); // configuration from Hadoop client + AppHandle appHandle = params.getApexLauncher().launchApp(apexApp, conf, + params.launchAttributes); + if (appHandle == null) { + throw new AssertionError("Launch returns null handle."); + } + // TODO (future PR) + // At this point the application is running, but this process should remain active to + // allow the parent to implement the runner result. + } + + /** + * Launch parameters that will be serialized and passed to the child process. + */ + @VisibleForTesting + protected static class LaunchParams implements Serializable { + private static final long serialVersionUID = 1L; + private final DAG dag; + private final Attribute.AttributeMap launchAttributes; + private HashMap env; + private String cmd; + + protected LaunchParams(DAG dag, AttributeMap launchAttributes) { + this.dag = dag; + this.launchAttributes = launchAttributes; + } + + protected Launcher getApexLauncher() { + return Launcher.getLauncher(LaunchMode.YARN); + } + + protected String getCmd() { + return cmd; + } + + protected Map getEnv() { + return env; + } + + } + + private static void copyShallow(DAG from, DAG to) { + checkArgument(from.getClass() == to.getClass(), "must be same class %s %s", + from.getClass(), to.getClass()); + Field[] fields = from.getClass().getDeclaredFields(); + AccessibleObject.setAccessible(fields, true); + for (int i = 0; i < fields.length; i++) { + Field field = fields[i]; + if (!java.lang.reflect.Modifier.isStatic(field.getModifiers())) { + try { + field.set(to, field.get(from)); + } catch (IllegalArgumentException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + } + + /** + * Starts a command and waits for it to complete. + */ + public static class ProcessWatcher implements Runnable { + private final Process p; + private volatile boolean finished = false; + private volatile int rc; + + public ProcessWatcher(Process p) { + this.p = p; + new Thread(this).start(); + } + + public boolean isFinished() { + return finished; + } + + @Override + public void run() { + try { + rc = p.waitFor(); + } catch (Exception e) { + // ignore + } + finished = true; + } + } + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java new file mode 100644 index 000000000000..986818bf8f1a --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java @@ -0,0 +1,138 @@ +/* + * 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.apex; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; + +import com.datatorrent.api.Attribute; +import com.datatorrent.api.Attribute.AttributeMap; +import com.datatorrent.api.Context.DAGContext; +import com.datatorrent.api.DAG; +import com.datatorrent.api.StreamingApplication; + +import java.io.File; +import java.net.URI; +import java.nio.file.FileSystem; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.jar.JarFile; + +import org.apache.apex.api.EmbeddedAppLauncher; +import org.apache.apex.api.Launcher; +import org.apache.apex.api.Launcher.AppHandle; +import org.apache.apex.api.Launcher.LaunchMode; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.junit.Assert; +import org.junit.Test; + +/** + * Test for dependency resolution for pipeline execution on YARN. + */ +public class ApexYarnLauncherTest { + + @Test + public void testGetYarnDeployDependencies() throws Exception { + List deps = ApexYarnLauncher.getYarnDeployDependencies(); + String depsToString = deps.toString(); + // the beam dependencies are not present as jar when running within the Maven build reactor + //assertThat(depsToString, containsString("beam-runners-core-")); + //assertThat(depsToString, containsString("beam-runners-apex-")); + assertThat(depsToString, containsString("apex-common-")); + assertThat(depsToString, not(containsString("hadoop-"))); + assertThat(depsToString, not(containsString("zookeeper-"))); + } + + @Test + public void testProxyLauncher() throws Exception { + // use the embedded launcher to build the DAG only + EmbeddedAppLauncher embeddedLauncher = Launcher.getLauncher(LaunchMode.EMBEDDED); + + StreamingApplication app = new StreamingApplication() { + @Override + public void populateDAG(DAG dag, Configuration conf) { + dag.setAttribute(DAGContext.APPLICATION_NAME, "DummyApp"); + } + }; + + Configuration conf = new Configuration(false); + DAG dag = embeddedLauncher.prepareDAG(app, conf); + Attribute.AttributeMap launchAttributes = new Attribute.AttributeMap.DefaultAttributeMap(); + ApexYarnLauncher launcher = new ApexYarnLauncher(); + launcher.launchApp(new MockApexYarnLauncherParams(dag, launchAttributes)); + } + + private static class MockApexYarnLauncherParams extends ApexYarnLauncher.LaunchParams { + private static final long serialVersionUID = 1L; + + public MockApexYarnLauncherParams(DAG dag, AttributeMap launchAttributes) { + super(dag, launchAttributes); + } + + @Override + protected Launcher getApexLauncher() { + return new Launcher() { + @Override + public AppHandle launchApp(StreamingApplication application, + Configuration configuration, AttributeMap launchParameters) + throws org.apache.apex.api.Launcher.LauncherException { + EmbeddedAppLauncher embeddedLauncher = Launcher.getLauncher(LaunchMode.EMBEDDED); + DAG dag = embeddedLauncher.getDAG(); + application.populateDAG(dag, new Configuration(false)); + String appName = dag.getValue(DAGContext.APPLICATION_NAME); + Assert.assertEquals("DummyApp", appName); + return new AppHandle() { + @Override + public boolean isFinished() { + return true; + } + @Override + public void shutdown(org.apache.apex.api.Launcher.ShutdownMode arg0) { + } + }; + } + }; + } + + } + + @Test + public void testCreateJar() throws Exception { + File baseDir = new File("./target/testCreateJar"); + File srcDir = new File(baseDir, "src"); + String file1 = "file1"; + FileUtils.forceMkdir(srcDir); + FileUtils.write(new File(srcDir, file1), "file1"); + + File jarFile = new File(baseDir, "test.jar"); + ApexYarnLauncher.createJar(srcDir, jarFile); + Assert.assertTrue("exists: " + jarFile, jarFile.exists()); + URI uri = URI.create("jar:" + jarFile.toURI()); + final Map env = Collections.singletonMap("create", "true"); + try (final FileSystem zipfs = FileSystems.newFileSystem(uri, env);) { + Assert.assertTrue("manifest", Files.isRegularFile(zipfs.getPath(JarFile.MANIFEST_NAME))); + Assert.assertTrue("file1", Files.isRegularFile(zipfs.getPath(file1))); + } + + } +} From 9ebc4653afffff39aa40217635dbe6f22450be78 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Wed, 7 Dec 2016 13:27:53 -0800 Subject: [PATCH 144/279] [BEAM-551] Fix handling of TextIO.Sink --- .../beam/runners/dataflow/DataflowRunner.java | 7 ++- .../runners/dataflow/DataflowRunnerTest.java | 20 ++++++ .../org/apache/beam/sdk/io/FileBasedSink.java | 62 ++++++++++++------- .../apache/beam/sdk/io/FileBasedSinkTest.java | 6 +- .../org/apache/beam/sdk/io/TextIOTest.java | 19 ++++++ .../org/apache/beam/sdk/io/XmlSinkTest.java | 4 +- 6 files changed, 89 insertions(+), 29 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 85318e6ad14a..d902ccbea178 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -2065,8 +2065,11 @@ public BatchWrite(DataflowRunner runner, Write.Bound transform) { public PDone expand(PCollection input) { if (transform.getSink() instanceof FileBasedSink) { FileBasedSink sink = (FileBasedSink) transform.getSink(); - PathValidator validator = runner.options.getPathValidator(); - validator.validateOutputFilePrefixSupported(sink.getBaseOutputFilename()); + if (sink.getBaseOutputFilenameProvider().isAccessible()) { + PathValidator validator = runner.options.getPathValidator(); + validator.validateOutputFilePrefixSupported( + sink.getBaseOutputFilenameProvider().get()); + } } return transform.expand(input); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 133ae8af8e0c..4159b611e0b4 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -82,6 +82,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.dataflow.TestCountingSource; import org.apache.beam.sdk.testing.ExpectedLogs; @@ -313,6 +314,25 @@ public void testRun() throws IOException { assertValidJob(jobCaptor.getValue()); } + /** Options for testing. */ + public interface RuntimeTestOptions extends PipelineOptions { + ValueProvider getInput(); + void setInput(ValueProvider value); + + ValueProvider getOutput(); + void setOutput(ValueProvider value); + } + + @Test + public void testTextIOWithRuntimeParameters() throws IOException { + DataflowPipelineOptions dataflowOptions = buildPipelineOptions(); + RuntimeTestOptions options = dataflowOptions.as(RuntimeTestOptions.class); + Pipeline p = buildDataflowPipeline(dataflowOptions); + p + .apply(TextIO.Read.from(options.getInput()).withoutValidation()) + .apply(TextIO.Write.to(options.getOutput()).withoutValidation()); + } + @Test public void testRunReturnDifferentRequestId() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index 1396ab65ab9f..33296b49ab44 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -42,7 +42,9 @@ import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; @@ -201,8 +203,8 @@ public FileBasedSink(ValueProvider baseOutputFilename, String extension, /** * Returns the base output filename for this file based sink. */ - public String getBaseOutputFilename() { - return baseOutputFilename.get(); + public ValueProvider getBaseOutputFilenameProvider() { + return baseOutputFilename; } @Override @@ -290,7 +292,7 @@ public abstract static class FileBasedWriteOperation extends WriteOperation sink; /** Directory for temporary output files. */ - protected final String tempDirectory; + protected final ValueProvider tempDirectory; /** Constructs a temporary file path given the temporary directory and a filename. */ protected static String buildTemporaryFilename(String tempDirectory, String filename) @@ -308,22 +310,31 @@ protected static String buildTemporaryFilename(String tempDirectory, String file * @param sink the FileBasedSink that will be used to configure this write operation. */ public FileBasedWriteOperation(FileBasedSink sink) { - this(sink, buildTemporaryDirectoryName(sink.getBaseOutputFilename())); + this(sink, NestedValueProvider.of( + sink.getBaseOutputFilenameProvider(), new TemporaryDirectoryBuilder())); } - private static String buildTemporaryDirectoryName(String baseOutputFilename) { - try { - IOChannelFactory factory = IOChannelUtils.getFactory(baseOutputFilename); - Path baseOutputPath = factory.toPath(baseOutputFilename); - return baseOutputPath - .resolveSibling( - "temp-beam-" - + baseOutputPath.getFileName() - + "-" - + Instant.now().toString(DateTimeFormat.forPattern("yyyy-MM-DD_HH-mm-ss"))) - .toString(); - } catch (IOException e) { - throw new RuntimeException(e); + private static class TemporaryDirectoryBuilder + implements SerializableFunction { + // The intent of the code is to have a consistent value of tempDirectory across + // all workers, which wouldn't happen if now() was called inline. + Instant now = Instant.now(); + + @Override + public String apply(String baseOutputFilename) { + try { + IOChannelFactory factory = IOChannelUtils.getFactory(baseOutputFilename); + Path baseOutputPath = factory.toPath(baseOutputFilename); + return baseOutputPath + .resolveSibling( + "temp-beam-" + + baseOutputPath.getFileName() + + "-" + + now.toString(DateTimeFormat.forPattern("yyyy-MM-DD_HH-mm-ss"))) + .toString(); + } catch (IOException e) { + throw new RuntimeException(e); + } } } @@ -334,6 +345,10 @@ private static String buildTemporaryDirectoryName(String baseOutputFilename) { * @param tempDirectory the base directory to be used for temporary output files. */ public FileBasedWriteOperation(FileBasedSink sink, String tempDirectory) { + this(sink, StaticValueProvider.of(tempDirectory)); + } + + private FileBasedWriteOperation(FileBasedSink sink, ValueProvider tempDirectory) { this.sink = sink; this.tempDirectory = tempDirectory; } @@ -452,8 +467,9 @@ protected final List generateDestinationFilenames(int numFiles) { */ protected final void removeTemporaryFiles(List knownFiles, PipelineOptions options) throws IOException { - LOG.debug("Removing temporary bundle output files in {}.", tempDirectory); - IOChannelFactory factory = IOChannelUtils.getFactory(tempDirectory); + String tempDir = tempDirectory.get(); + LOG.debug("Removing temporary bundle output files in {}.", tempDir); + IOChannelFactory factory = IOChannelUtils.getFactory(tempDir); // To partially mitigate the effects of filesystems with eventually-consistent // directory matching APIs, we remove not only files that the filesystem says exist @@ -461,17 +477,17 @@ protected final void removeTemporaryFiles(List knownFiles, PipelineOptio // (produced by successfully completed bundles). // This may still fail to remove temporary outputs of some failed bundles, but at least // the common case (where all bundles succeed) is guaranteed to be fully addressed. - Collection matches = factory.match(factory.resolve(tempDirectory, "*")); + Collection matches = factory.match(factory.resolve(tempDir, "*")); Set allMatches = new HashSet<>(matches); allMatches.addAll(knownFiles); LOG.debug( "Removing {} temporary files found under {} ({} matched glob, {} known files)", allMatches.size(), - tempDirectory, + tempDir, matches.size(), allMatches.size() - matches.size()); factory.remove(allMatches); - factory.remove(ImmutableList.of(tempDirectory)); + factory.remove(ImmutableList.of(tempDir)); } /** @@ -569,7 +585,7 @@ protected void writeFooter() throws Exception {} public final void open(String uId) throws Exception { this.id = uId; filename = FileBasedWriteOperation.buildTemporaryFilename( - getWriteOperation().tempDirectory, uId); + getWriteOperation().tempDirectory.get(), uId); LOG.debug("Opening {}.", filename); final WritableByteChannelFactory factory = getWriteOperation().getSink().writableByteChannelFactory; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java index 4ab384366954..930ca29fb201 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java @@ -231,7 +231,9 @@ private void runFinalize(SimpleSink.SimpleWriteOperation writeOp, List tem assertFalse(temporaryFiles.get(i).exists()); } - assertFalse(new File(writeOp.tempDirectory).exists()); + assertFalse(new File(writeOp.tempDirectory.get()).exists()); + // Test that repeated requests of the temp directory return a stable result. + assertEquals(writeOp.tempDirectory.get(), writeOp.tempDirectory.get()); } /** @@ -487,7 +489,7 @@ public void testFileBasedWriterWithWritableByteChannelFactory() throws Exception final FileBasedWriter writer = writeOp.createWriter(null); final String expectedFilename = - writeOp.tempDirectory + "/" + testUid; + writeOp.tempDirectory.get() + "/" + testUid; final List expected = new ArrayList<>(); expected.add("header"); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index d3a5d5e6e661..472399a9d4e2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -88,6 +88,7 @@ import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -629,6 +630,24 @@ public void testBadWildcardRecursive() throws Exception { pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz")); } + /** Options for testing. */ + public interface RuntimeTestOptions extends PipelineOptions { + ValueProvider getInput(); + void setInput(ValueProvider value); + + ValueProvider getOutput(); + void setOutput(ValueProvider value); + } + + @Test + public void testRuntimeOptionsNotCalledInApply() throws Exception { + Pipeline pipeline = TestPipeline.create(); + RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); + pipeline + .apply(TextIO.Read.from(options.getInput()).withoutValidation()) + .apply(TextIO.Write.to(options.getOutput()).withoutValidation()); + } + @Test public void testReadWithoutValidationFlag() throws Exception { TextIO.Read.Bound read = TextIO.Read.from("gs://bucket/foo*/baz"); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java index f9a9655e1646..96b8c5733b5d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java @@ -146,7 +146,7 @@ public void testCreateWriteOperations() { assertEquals(testRootElement, writeOp.getSink().rootElementName); assertEquals(XmlSink.XML_EXTENSION, writeOp.getSink().extension); Path outputPath = new File(testFilePrefix).toPath(); - Path tempPath = new File(writeOp.tempDirectory).toPath(); + Path tempPath = new File(writeOp.tempDirectory.get()).toPath(); assertEquals(outputPath.getParent(), tempPath.getParent()); assertThat( tempPath.getFileName().toString(), containsString("temp-beam-" + outputPath.getFileName())); @@ -163,7 +163,7 @@ public void testCreateWriter() throws Exception { .createWriteOperation(options); XmlWriter writer = writeOp.createWriter(options); Path outputPath = new File(testFilePrefix).toPath(); - Path tempPath = new File(writer.getWriteOperation().tempDirectory).toPath(); + Path tempPath = new File(writer.getWriteOperation().tempDirectory.get()).toPath(); assertEquals(outputPath.getParent(), tempPath.getParent()); assertThat( tempPath.getFileName().toString(), containsString("temp-beam-" + outputPath.getFileName())); From aba40e2de9ba058f33086eb6a913fa583a82b058 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Thu, 8 Dec 2016 15:07:06 +0200 Subject: [PATCH 145/279] [BEAM-921] spark-runner: register sources and coders to serialize with java serializer --- runners/spark/pom.xml | 35 +++++------ .../coders/BeamSparkRunnerRegistrator.java | 60 ++++++++++++++----- .../BeamSparkRunnerRegistratorTest.java | 57 ++++++++++++++++++ 3 files changed, 118 insertions(+), 34 deletions(-) create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistratorTest.java diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index d1ef225ceeb5..86e9039a80c8 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -54,7 +54,7 @@ - local-runnable-on-service-tests false @@ -134,28 +134,14 @@ ${hadoop.version} provided + com.esotericsoftware.kryo kryo - 2.21 - provided - - - de.javakaffee - kryo-serializers - 0.39 - - - - com.esotericsoftware - kryo - - - - com.google.protobuf - protobuf-java - - + 2.21.1 com.google.code.findbugs @@ -264,6 +250,11 @@ metrics-core ${dropwizard.metrics.version} + + org.reflections + reflections + 0.9.10 + @@ -405,6 +396,10 @@ com.google.thirdparty org.apache.beam.spark.relocated.com.google.thirdparty + + com.esotericsoftware.kryo + org.apache.beam.spark.relocated.com.esotericsoftware.kryo + true spark-app diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java index 0e627815b883..41b0a0198f6c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistrator.java @@ -19,28 +19,60 @@ package org.apache.beam.runners.spark.coders; import com.esotericsoftware.kryo.Kryo; -import de.javakaffee.kryoserializers.UnmodifiableCollectionsSerializer; -import de.javakaffee.kryoserializers.guava.ImmutableListSerializer; -import de.javakaffee.kryoserializers.guava.ImmutableMapSerializer; -import de.javakaffee.kryoserializers.guava.ImmutableMultimapSerializer; -import de.javakaffee.kryoserializers.guava.ImmutableSetSerializer; -import de.javakaffee.kryoserializers.guava.ReverseListSerializer; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.Source; import org.apache.spark.serializer.KryoRegistrator; +import org.reflections.Reflections; /** - * Custom {@link com.esotericsoftware.kryo.Serializer}s for Beam's Spark runner needs. + * Custom {@link KryoRegistrator}s for Beam's Spark runner needs. */ public class BeamSparkRunnerRegistrator implements KryoRegistrator { @Override public void registerClasses(Kryo kryo) { - UnmodifiableCollectionsSerializer.registerSerializers(kryo); - // Guava - ImmutableListSerializer.registerSerializers(kryo); - ImmutableSetSerializer.registerSerializers(kryo); - ImmutableMapSerializer.registerSerializers(kryo); - ImmutableMultimapSerializer.registerSerializers(kryo); - ReverseListSerializer.registerSerializers(kryo); + for (Class clazz : ClassesForJavaSerialization.getClasses()) { + kryo.register(clazz, new JavaSerializer()); + } + } + + /** + * Register coders and sources with {@link JavaSerializer} since they aren't guaranteed to be + * Kryo-serializable. + */ + private static class ClassesForJavaSerialization { + private static final Class[] CLASSES_FOR_JAVA_SERIALIZATION = new Class[]{ + Coder.class, Source.class + }; + + private static final Iterable> INSTANCE; + + /** + * Find all subclasses of ${@link CLASSES_FOR_JAVA_SERIALIZATION} + */ + static { + final Reflections reflections = new Reflections(); + INSTANCE = Iterables.concat(Lists.transform(Arrays.asList(CLASSES_FOR_JAVA_SERIALIZATION), + new Function>>() { + @SuppressWarnings({"unchecked", "ConstantConditions"}) + @Nullable + @Override + public Set> apply(@Nullable Class clazz) { + return reflections.getSubTypesOf(clazz); + } + })); + } + + static Iterable> getClasses() { + return INSTANCE; + } } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistratorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistratorTest.java new file mode 100644 index 000000000000..e35301750cf6 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/BeamSparkRunnerRegistratorTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark.coders; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.collect.Iterables; +import java.io.Serializable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.Source; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.reflections.Reflections; + + +/** + * BeamSparkRunnerRegistrator Test. + */ +public class BeamSparkRunnerRegistratorTest { + @Test + public void testCodersAndSourcesRegistration() { + BeamSparkRunnerRegistrator registrator = new BeamSparkRunnerRegistrator(); + + Reflections reflections = new Reflections(); + Iterable> classesForJavaSerialization = + Iterables.concat(reflections.getSubTypesOf(Coder.class), + reflections.getSubTypesOf(Source.class)); + + Kryo kryo = new Kryo(); + + registrator.registerClasses(kryo); + + for (Class clazz : classesForJavaSerialization) { + Assert.assertThat("Registered serializer for class " + clazz.getName() + + " was not an instance of " + JavaSerializer.class.getName(), + kryo.getSerializer(clazz), + Matchers.instanceOf(JavaSerializer.class)); + } + } +} From 36f09c4d13f76d477e3575d8ee5bbfb1825c69ab Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sat, 10 Dec 2016 15:59:54 -0800 Subject: [PATCH 146/279] [BEAM-551] Fix toString for FileBasedSource --- .../org/apache/beam/sdk/io/FileBasedSource.java | 6 ++++-- .../apache/beam/sdk/io/FileBasedSourceTest.java | 14 ++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java index e0fc6b60039e..d835f9b9cd15 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java @@ -411,11 +411,13 @@ public final BoundedReader createReader(PipelineOptions options) throws IOExc @Override public String toString() { + String fileString = fileOrPatternSpec.isAccessible() + ? fileOrPatternSpec.get() : fileOrPatternSpec.toString(); switch (mode) { case FILEPATTERN: - return fileOrPatternSpec.toString(); + return fileString; case SINGLE_FILE_OR_SUBRANGE: - return fileOrPatternSpec.toString() + " range " + super.toString(); + return fileString + " range " + super.toString(); default: throw new IllegalStateException("Unexpected mode: " + mode); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java index dde5d02b1e83..a065191cb317 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java @@ -916,4 +916,18 @@ public void testSplitAtFractionExhaustive() throws Exception { TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 1, 0, file.length(), null); assertSplitAtFractionExhaustive(source, options); } + + @Test + public void testToStringFile() throws Exception { + String path = "/tmp/foo"; + TestFileBasedSource source = new TestFileBasedSource(path, 1, 0, 10, null); + assertEquals(String.format("%s range [0, 10)", path), source.toString()); + } + + @Test + public void testToStringPattern() throws Exception { + String path = "/tmp/foo/*"; + TestFileBasedSource source = new TestFileBasedSource(path, 1, 0, 10, null); + assertEquals(String.format("%s range [0, 10)", path), source.toString()); + } } From cfcfa2f3e739a3a71b1ec9edf31f8023e1a5ed3f Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 9 Dec 2016 18:35:52 +0800 Subject: [PATCH 147/279] [BEAM-1120] Move some DataflowRunner configurations from code to properties --- .../beam/runners/dataflow/DataflowRunner.java | 14 +-- .../runners/dataflow/DataflowRunnerInfo.java | 92 +++++++++++++++++++ .../DataflowPipelineWorkerPoolOptions.java | 6 +- .../beam/runners/dataflow/dataflow.properties | 23 +++++ .../dataflow/DataflowRunnerInfoTest.java | 51 ++++++++++ .../org/apache/beam/sdk/util/ReleaseInfo.java | 4 - 6 files changed, 172 insertions(+), 18 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java create mode 100644 runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties create mode 100644 runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index d902ccbea178..711b1b0d0555 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -205,16 +205,6 @@ public class DataflowRunner extends PipelineRunner { /** A set of user defined functions to invoke at different points in execution. */ private DataflowRunnerHooks hooks; - // Environment version information. - private static final String ENVIRONMENT_MAJOR_VERSION = "6"; - - // Default Docker container images that execute Dataflow worker harness, residing in Google - // Container Registry, separately for Batch and Streaming. - public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161205"; - public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161205"; - // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; @@ -546,7 +536,9 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Requirements about the service. Map environmentVersion = new HashMap<>(); - environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION); + environmentVersion.put( + PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, + DataflowRunnerInfo.getDataflowRunnerInfo().getEnvironmentMajorVersion()); newJob.getEnvironment().setVersion(environmentVersion); // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can // autoscale if specified. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java new file mode 100644 index 000000000000..59cb8a49de26 --- /dev/null +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java @@ -0,0 +1,92 @@ +/* + * 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.dataflow; + +import static com.google.common.base.Preconditions.checkState; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Properties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Populates versioning and other information for {@link DataflowRunner}. + */ +public final class DataflowRunnerInfo { + private static final Logger LOG = LoggerFactory.getLogger(DataflowRunnerInfo.class); + + private static final String PROPERTIES_PATH = + "/org/apache/beam/runners/dataflow/dataflow.properties"; + + private static class LazyInit { + private static final DataflowRunnerInfo INSTANCE = new DataflowRunnerInfo(PROPERTIES_PATH); + } + + /** + * Returns an instance of {@link DataflowRunnerInfo}. + */ + public static DataflowRunnerInfo getDataflowRunnerInfo() { + return LazyInit.INSTANCE; + } + + private Properties properties; + + private static final String ENVIRONMENT_MAJOR_VERSION_KEY = "environment.major.version"; + private static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY = "worker.image.batch"; + private static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY = + "worker.image.streaming"; + + /** Provides the environment's major version number. */ + public String getEnvironmentMajorVersion() { + checkState( + properties.containsKey(ENVIRONMENT_MAJOR_VERSION_KEY), "Unknown environment major version"); + return properties.getProperty(ENVIRONMENT_MAJOR_VERSION_KEY); + } + + /** Provides the batch worker harness container image name. */ + public String getBatchWorkerHarnessContainerImage() { + checkState( + properties.containsKey(BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY), + "Unknown batch worker harness container image"); + return properties.getProperty(BATCH_WORKER_HARNESS_CONTAINER_IMAGE_KEY); + } + + /** Provides the streaming worker harness container image name. */ + public String getStreamingWorkerHarnessContainerImage() { + checkState( + properties.containsKey(STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY), + "Unknown streaming worker harness container image"); + return properties.getProperty(STREAMING_WORKER_HARNESS_CONTAINER_IMAGE_KEY); + } + + private DataflowRunnerInfo(String resourcePath) { + properties = new Properties(); + + try (InputStream in = DataflowRunnerInfo.class.getResourceAsStream(PROPERTIES_PATH)) { + if (in == null) { + LOG.warn("Dataflow runner properties resource not found: {}", resourcePath); + return; + } + + properties.load(in); + } catch (IOException e) { + LOG.warn("Error loading Dataflow runner properties resource: ", e); + } + } +} diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java index 157321a13e2a..05086b0d8901 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java @@ -20,7 +20,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.List; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.runners.dataflow.DataflowRunnerInfo; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; @@ -130,9 +130,9 @@ class WorkerHarnessContainerImageFactory public String create(PipelineOptions options) { DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); if (dataflowOptions.isStreaming()) { - return DataflowRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE; + return DataflowRunnerInfo.getDataflowRunnerInfo().getStreamingWorkerHarnessContainerImage(); } else { - return DataflowRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE; + return DataflowRunnerInfo.getDataflowRunnerInfo().getBatchWorkerHarnessContainerImage(); } } } diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties new file mode 100644 index 000000000000..1eae8cb9f055 --- /dev/null +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -0,0 +1,23 @@ +# +# 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. +# +# Dataflow runtime properties + +environment.major.version=6 + +worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161205 + +worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161205 diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java new file mode 100644 index 000000000000..9b5b37497367 --- /dev/null +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.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.dataflow; + +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +/** + * Tests for {@link DataflowRunnerInfo}. + */ +public class DataflowRunnerInfoTest { + + @Test + public void getDataflowRunnerInfo() throws Exception { + DataflowRunnerInfo info = DataflowRunnerInfo.getDataflowRunnerInfo(); + + String version = info.getEnvironmentMajorVersion(); + // Validate major version is a number + assertTrue( + String.format("Environment major version number %s is not a number", version), + version.matches("\\d+")); + + // Validate container images contain gcr.io + assertThat( + "batch worker harness container image invalid", + info.getBatchWorkerHarnessContainerImage(), + containsString("gcr.io")); + assertThat( + "streaming worker harness container image invalid", + info.getStreamingWorkerHarnessContainerImage(), + containsString("gcr.io")); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java index ba80de97dc4d..eeac5571c316 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java @@ -74,10 +74,6 @@ private ReleaseInfo(String resourcePath) { } for (String name : properties.stringPropertyNames()) { - if (name.equals("name")) { - // We don't allow the properties to override the SDK name. - continue; - } put(name, properties.getProperty(name)); } } From 66c29e4d8fd3654899fed6dc0054194f9e6a9b74 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sat, 10 Dec 2016 09:16:57 -0800 Subject: [PATCH 148/279] [BEAM-551] Fix handling of default for VP --- .../org/apache/beam/sdk/options/ValueProvider.java | 13 ++++++++++--- .../apache/beam/sdk/options/ValueProviderTest.java | 12 ++++++++++++ 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java index 3d36a29603b9..93fcaf898048 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.options; -import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.core.JsonGenerator; @@ -222,8 +221,16 @@ public T get() { Method method = klass.getMethod(methodName); PipelineOptions methodOptions = options.as(klass); InvocationHandler handler = Proxy.getInvocationHandler(methodOptions); - T value = ((ValueProvider) handler.invoke(methodOptions, method, null)).get(); - return firstNonNull(value, defaultValue); + ValueProvider result = + (ValueProvider) handler.invoke(methodOptions, method, null); + // Two cases: If we have deserialized a new value from JSON, it will + // be wrapped in a StaticValueProvider, which we can provide here. If + // not, there was no JSON value, and we return the default, whether or + // not it is null. + if (result instanceof StaticValueProvider) { + return result.get(); + } + return defaultValue; } catch (Throwable e) { throw new RuntimeException("Unable to load runtime value.", e); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java index 7ec40be21310..ea5cc54e7dcf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java @@ -149,6 +149,18 @@ public void testDefaultRuntimeProviderWithOverride() throws Exception { assertEquals("quux", provider.get()); } + @Test + public void testDefaultRuntimeProviderWithoutOverride() throws Exception { + TestOptions runtime = PipelineOptionsFactory.as(TestOptions.class); + TestOptions options = PipelineOptionsFactory.as(TestOptions.class); + runtime.setOptionsId(options.getOptionsId()); + RuntimeValueProvider.setRuntimeOptions(runtime); + + ValueProvider provider = options.getBar(); + assertTrue(provider.isAccessible()); + assertEquals("bar", provider.get()); + } + /** A test interface. */ public interface BadOptionsRuntime extends PipelineOptions { RuntimeValueProvider getBar(); From a47eac91c70846d2aa3a945e327e2b148b16ca5f Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Wed, 7 Dec 2016 15:31:52 -0800 Subject: [PATCH 149/279] Fix handling of null ValueProviders in DisplayData --- .../apache/beam/sdk/transforms/display/DisplayData.java | 8 +++++++- .../beam/sdk/transforms/display/DisplayDataTest.java | 2 ++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java index f0040f75b8b1..d3bfe93e22e8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java @@ -866,9 +866,15 @@ public static ItemSpec item(String key, @Nullable String value) { /** * Create a display item for the specified key and {@link ValueProvider}. */ - public static ItemSpec item(String key, ValueProvider value) { + public static ItemSpec item(String key, @Nullable ValueProvider value) { + if (value == null) { + return item(key, Type.STRING, null); + } if (value.isAccessible()) { Object got = value.get(); + if (got == null) { + return item(key, Type.STRING, null); + } Type type = inferType(got); if (type == null) { throw new RuntimeException(String.format("Unknown value type: %s", got)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index f5c1e73fd76e..06b2bce0372e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -379,6 +379,8 @@ public void testAddIfNotNull() { public void populateDisplayData(Builder builder) { builder .addIfNotNull(DisplayData.item("nullString", (String) null)) + .addIfNotNull(DisplayData.item("nullVPString", (ValueProvider) null)) + .addIfNotNull(DisplayData.item("nullierVPString", StaticValueProvider.of(null))) .addIfNotNull(DisplayData.item("notNullString", "foo")) .addIfNotNull(DisplayData.item("nullLong", (Long) null)) .addIfNotNull(DisplayData.item("notNullLong", 1234L)) From fd6d09c32f6bcf67c63ec74548373ee90d67f2bd Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Sun, 4 Dec 2016 14:16:23 -0800 Subject: [PATCH 150/279] BigQueryIO.Write: support runtime schema and table --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 217 ++++++++++++------ .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 60 ++++- 2 files changed, 206 insertions(+), 71 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index f99ca7869613..0be8567fe5fd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -321,6 +321,23 @@ public static String toTableSpec(TableReference ref) { return sb.toString(); } + @VisibleForTesting + static class JsonSchemaToTableSchema + implements SerializableFunction { + @Override + public TableSchema apply(String from) { + return fromJsonString(from, TableSchema.class); + } + } + + private static class TableSchemaToJsonSchema + implements SerializableFunction { + @Override + public String apply(TableSchema from) { + return toJsonString(from); + } + } + private static class JsonTableRefToTableRef implements SerializableFunction { @Override @@ -329,6 +346,14 @@ public TableReference apply(String from) { } } + private static class TableRefToTableSpec + implements SerializableFunction { + @Override + public String apply(TableReference from) { + return toTableSpec(from); + } + } + private static class TableRefToJson implements SerializableFunction { @Override @@ -353,6 +378,15 @@ public TableReference apply(String from) { } } + @Nullable + private static ValueProvider displayTable( + @Nullable ValueProvider table) { + if (table == null) { + return null; + } + return NestedValueProvider.of(table, new TableRefToTableSpec()); + } + /** * A {@link PTransform} that reads from a BigQuery table and returns a * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table. @@ -659,11 +693,11 @@ public PCollection expand(PBegin input) { .setProjectId(executingProject) .setDatasetId(queryTempDatasetId) .setTableId(queryTempTableId); + String jsonTableRef = toJsonString(queryTempTableRef); source = BigQueryQuerySource.create( jobIdToken, query, NestedValueProvider.of( - StaticValueProvider.of( - toJsonString(queryTempTableRef)), new JsonTableRefToTableRef()), + StaticValueProvider.of(jsonTableRef), new JsonTableRefToTableRef()), flattenResults, useLegacySql, extractDestinationDir, bqServices); } else { ValueProvider inputTable = getTableWithDefaultProject(bqOptions); @@ -712,17 +746,10 @@ protected Coder getDefaultOutputCoder() { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - TableReference table = getTable(); - - if (table != null) { - builder.add(DisplayData.item("table", toTableSpec(table)) - .withLabel("Table")); - } - String queryString = query == null - ? null : query.isAccessible() - ? query.get() : query.toString(); builder - .addIfNotNull(DisplayData.item("query", queryString) + .addIfNotNull(DisplayData.item("table", displayTable(getTableProvider())) + .withLabel("Table")) + .addIfNotNull(DisplayData.item("query", query) .withLabel("Query")) .addIfNotNull(DisplayData.item("flattenResults", flattenResults) .withLabel("Flatten Query Results")) @@ -752,10 +779,10 @@ public void populateDisplayData(DisplayData.Builder builder) { if (Strings.isNullOrEmpty(table.get().getProjectId())) { // If user does not specify a project we assume the table to be located in // the default project. - TableReference ref = table.get(); - ref.setProjectId(bqOptions.getProject()); + TableReference tableRef = table.get(); + tableRef.setProjectId(bqOptions.getProject()); return NestedValueProvider.of(StaticValueProvider.of( - toJsonString(ref)), new JsonTableRefToTableRef()); + toJsonString(tableRef)), new JsonTableRefToTableRef()); } return table; } @@ -941,8 +968,7 @@ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - String table = jsonTable.isAccessible() ? jsonTable.get() : jsonTable.toString(); - builder.add(DisplayData.item("table", table)); + builder.add(DisplayData.item("table", jsonTable)); } } @@ -1060,7 +1086,7 @@ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("query", query.get())); + builder.add(DisplayData.item("query", query)); } private synchronized JobStatistics dryRunQueryIfNeeded(BigQueryOptions bqOptions) @@ -1515,6 +1541,11 @@ public static Bound to(String tableSpec) { return new Bound().to(tableSpec); } + /** Creates a write transformation for the given table. */ + public static Bound to(ValueProvider tableSpec) { + return new Bound().to(tableSpec); + } + /** Creates a write transformation for the given table. */ public static Bound to(TableReference table) { return new Bound().to(table); @@ -1558,6 +1589,13 @@ public static Bound withSchema(TableSchema schema) { return new Bound().withSchema(schema); } + /** + * Like {@link #withSchema(TableSchema)}, but with a {@link ValueProvider}. + */ + public static Bound withSchema(ValueProvider schema) { + return new Bound().withSchema(schema); + } + /** Creates a write transformation with the specified options for creating the table. */ public static Bound withCreateDisposition(CreateDisposition disposition) { return new Bound().withCreateDisposition(disposition); @@ -1593,12 +1631,12 @@ public static class Bound extends PTransform, PDone> { // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery job finishes. static final int LOAD_JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; - @Nullable final String jsonTableRef; + @Nullable final ValueProvider jsonTableRef; @Nullable final SerializableFunction tableRefFunction; // Table schema. The schema is required only if the table does not exist. - @Nullable final String jsonSchema; + @Nullable final ValueProvider jsonSchema; // Options for creating the table. Valid values are CREATE_IF_NEEDED and // CREATE_NEVER. @@ -1645,9 +1683,9 @@ public Bound() { null /* bigQueryServices */); } - private Bound(String name, @Nullable String jsonTableRef, + private Bound(String name, @Nullable ValueProvider jsonTableRef, @Nullable SerializableFunction tableRefFunction, - @Nullable String jsonSchema, + @Nullable ValueProvider jsonSchema, CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate, @Nullable BigQueryServices bigQueryServices) { super(name); @@ -1667,7 +1705,8 @@ private Bound(String name, @Nullable String jsonTableRef, *

      Does not modify this object. */ public Bound to(String tableSpec) { - return to(parseTableSpec(tableSpec)); + return toTableRef(NestedValueProvider.of( + StaticValueProvider.of(tableSpec), new TableSpecToTableRef())); } /** @@ -1676,7 +1715,28 @@ public Bound to(String tableSpec) { *

      Does not modify this object. */ public Bound to(TableReference table) { - return new Bound(name, toJsonString(table), tableRefFunction, jsonSchema, createDisposition, + return to(StaticValueProvider.of(toTableSpec(table))); + } + + /** + * Returns a copy of this write transformation, but writing to the specified table. Refer to + * {@link #parseTableSpec(String)} for the specification format. + * + *

      Does not modify this object. + */ + public Bound to(ValueProvider tableSpec) { + return toTableRef(NestedValueProvider.of(tableSpec, new TableSpecToTableRef())); + } + + /** + * Returns a copy of this write transformation, but writing to the specified table. + * + *

      Does not modify this object. + */ + private Bound toTableRef(ValueProvider table) { + return new Bound(name, + NestedValueProvider.of(table, new TableRefToJson()), + tableRefFunction, jsonSchema, createDisposition, writeDisposition, validate, bigQueryServices); } @@ -1716,7 +1776,17 @@ public Bound toTableReference( *

      Does not modify this object. */ public Bound withSchema(TableSchema schema) { - return new Bound(name, jsonTableRef, tableRefFunction, toJsonString(schema), + return new Bound(name, jsonTableRef, tableRefFunction, + StaticValueProvider.of(toJsonString(schema)), + createDisposition, writeDisposition, validate, bigQueryServices); + } + + /** + * Like {@link #withSchema(TableSchema)}, but with a {@link ValueProvider}. + */ + public Bound withSchema(ValueProvider schema) { + return new Bound(name, jsonTableRef, tableRefFunction, + NestedValueProvider.of(schema, new TableSchemaToJsonSchema()), createDisposition, writeDisposition, validate, bigQueryServices); } @@ -1798,7 +1868,7 @@ public void validate(PCollection input) { // The user specified a table. if (jsonTableRef != null && validate) { - TableReference table = getTableWithDefaultProject(options); + TableReference table = getTableWithDefaultProject(options).get(); DatasetService datasetService = getBigQueryServices().getDatasetService(options); // Check for destination table presence and emptiness for early failure notification. @@ -1855,10 +1925,11 @@ public PDone expand(PCollection input) { // StreamWithDeDup and BigQuery's streaming import API. if (input.isBounded() == IsBounded.UNBOUNDED || tableRefFunction != null) { return input.apply( - new StreamWithDeDup(getTable(), tableRefFunction, getSchema(), bqServices)); + new StreamWithDeDup(getTable(), tableRefFunction, + NestedValueProvider.of(jsonSchema, new JsonSchemaToTableSchema()), bqServices)); } - TableReference table = getTableWithDefaultProject(options); + ValueProvider table = getTableWithDefaultProject(options); String jobIdToken = "beam_job_" + randomUUIDString(); String tempLocation = options.getTempLocation(); @@ -1909,7 +1980,7 @@ public PDone expand(PCollection input) { bqServices, jobIdToken, tempFilePrefix, - toJsonString(table), + NestedValueProvider.of(table, new TableRefToJson()), jsonSchema, WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED))); @@ -1920,7 +1991,7 @@ public PDone expand(PCollection input) { .of(new WriteRename( bqServices, jobIdToken, - toJsonString(table), + NestedValueProvider.of(table, new TableRefToJson()), writeDisposition, createDisposition, tempTablesView)) @@ -1934,7 +2005,7 @@ public PDone expand(PCollection input) { bqServices, jobIdToken, tempFilePrefix, - toJsonString(table), + NestedValueProvider.of(table, new TableRefToJson()), jsonSchema, writeDisposition, createDisposition))); @@ -2031,7 +2102,8 @@ public WriteDisposition getWriteDisposition() { /** Returns the table schema. */ public TableSchema getSchema() { - return fromJsonString(jsonSchema, TableSchema.class); + return fromJsonString( + jsonSchema == null ? null : jsonSchema.get(), TableSchema.class); } /** @@ -2039,20 +2111,32 @@ public TableSchema getSchema() { * *

      If the table's project is not specified, use the executing project. */ - @Nullable private TableReference getTableWithDefaultProject(BigQueryOptions bqOptions) { - TableReference table = getTable(); - if (table != null && Strings.isNullOrEmpty(table.getProjectId())) { + @Nullable private ValueProvider getTableWithDefaultProject( + BigQueryOptions bqOptions) { + ValueProvider table = getTable(); + if (table == null) { + return table; + } + if (!table.isAccessible()) { + LOG.info("Using a dynamic value for table input. This must contain a project" + + " in the table reference: {}", table); + return table; + } + if (Strings.isNullOrEmpty(table.get().getProjectId())) { // If user does not specify a project we assume the table to be located in // the default project. - table.setProjectId(bqOptions.getProject()); + TableReference tableRef = table.get(); + tableRef.setProjectId(bqOptions.getProject()); + return NestedValueProvider.of(StaticValueProvider.of( + toJsonString(tableRef)), new JsonTableRefToTableRef()); } return table; } /** Returns the table reference, or {@code null}. */ @Nullable - public TableReference getTable() { - return fromJsonString(jsonTableRef, TableReference.class); + public ValueProvider getTable() { + return NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableRef()); } /** Returns {@code true} if table validation is enabled. */ @@ -2172,8 +2256,8 @@ static class WriteTables extends DoFn>>, String> private final BigQueryServices bqServices; private final String jobIdToken; private final String tempFilePrefix; - private final String jsonTableRef; - private final String jsonSchema; + private final ValueProvider jsonTableRef; + private final ValueProvider jsonSchema; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; @@ -2182,8 +2266,8 @@ public WriteTables( BigQueryServices bqServices, String jobIdToken, String tempFilePrefix, - String jsonTableRef, - String jsonSchema, + ValueProvider jsonTableRef, + ValueProvider jsonSchema, WriteDisposition writeDisposition, CreateDisposition createDisposition) { this.singlePartition = singlePartition; @@ -2200,7 +2284,7 @@ public WriteTables( public void processElement(ProcessContext c) throws Exception { List partition = Lists.newArrayList(c.element().getValue()).get(0); String jobIdPrefix = String.format(jobIdToken + "_%05d", c.element().getKey()); - TableReference ref = fromJsonString(jsonTableRef, TableReference.class); + TableReference ref = fromJsonString(jsonTableRef.get(), TableReference.class); if (!singlePartition) { ref.setTableId(jobIdPrefix); } @@ -2209,7 +2293,8 @@ public void processElement(ProcessContext c) throws Exception { bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdPrefix, ref, - fromJsonString(jsonSchema, TableSchema.class), + fromJsonString( + jsonSchema == null ? null : jsonSchema.get(), TableSchema.class), partition, writeDisposition, createDisposition); @@ -2242,16 +2327,15 @@ private void load( .setProjectId(projectId) .setJobId(jobId); jobService.startLoadJob(jobRef, loadConfig); - Job job = jobService.pollJob(jobRef, Bound.LOAD_JOB_POLL_MAX_RETRIES); - Status jobStatus = parseStatus(job); + Status jobStatus = + parseStatus(jobService.pollJob(jobRef, Bound.LOAD_JOB_POLL_MAX_RETRIES)); switch (jobStatus) { case SUCCEEDED: return; case UNKNOWN: throw new RuntimeException("Failed to poll the load job status of job " + jobId); case FAILED: - LOG.info("BigQuery load job failed. Status: {} Details: {}", - jobId, job.getStatus()); + LOG.info("BigQuery load job failed: {}", jobId); continue; default: throw new IllegalStateException(String.format("Unexpected job status: %s of job %s", @@ -2306,7 +2390,7 @@ public void populateDisplayData(DisplayData.Builder builder) { static class WriteRename extends DoFn { private final BigQueryServices bqServices; private final String jobIdToken; - private final String jsonTableRef; + private final ValueProvider jsonTableRef; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; private final PCollectionView> tempTablesView; @@ -2314,7 +2398,7 @@ static class WriteRename extends DoFn { public WriteRename( BigQueryServices bqServices, String jobIdToken, - String jsonTableRef, + ValueProvider jsonTableRef, WriteDisposition writeDisposition, CreateDisposition createDisposition, PCollectionView> tempTablesView) { @@ -2342,7 +2426,7 @@ public void processElement(ProcessContext c) throws Exception { copy( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdToken, - fromJsonString(jsonTableRef, TableReference.class), + fromJsonString(jsonTableRef.get(), TableReference.class), tempTables, writeDisposition, createDisposition); @@ -2475,7 +2559,7 @@ private static void verifyTablePresence(DatasetService datasetService, TableRefe private static class StreamingWriteFn extends DoFn, TableRowInfo>, Void> { /** TableSchema in JSON. Use String to make the class Serializable. */ - private final String jsonTableSchema; + private final ValueProvider jsonTableSchema; private final BigQueryServices bqServices; @@ -2495,8 +2579,9 @@ private static class StreamingWriteFn createAggregator("ByteCount", new Sum.SumLongFn()); /** Constructor. */ - StreamingWriteFn(TableSchema schema, BigQueryServices bqServices) { - this.jsonTableSchema = toJsonString(schema); + StreamingWriteFn(ValueProvider schema, BigQueryServices bqServices) { + this.jsonTableSchema = + NestedValueProvider.of(schema, new TableSchemaToJsonSchema()); this.bqServices = checkNotNull(bqServices, "bqServices"); } @@ -2549,7 +2634,8 @@ public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec // check again. This check isn't needed for correctness, but we add it to prevent // every thread from attempting a create and overwhelming our BigQuery quota. if (!createdTables.contains(tableSpec)) { - TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class); + TableSchema tableSchema = JSON_FACTORY.fromString( + jsonTableSchema.get(), TableSchema.class); Bigquery client = Transport.newBigQueryClient(options).build(); BigQueryTableInserter inserter = new BigQueryTableInserter(client, options); inserter.getOrCreateTable(tableReference, Write.WriteDisposition.WRITE_APPEND, @@ -2708,7 +2794,7 @@ private static class TableRowInfo { private static class TagWithUniqueIdsAndTable extends DoFn, TableRowInfo>> { /** TableSpec to write to. */ - private final String tableSpec; + private final ValueProvider tableSpec; /** User function mapping windows to {@link TableReference} in JSON. */ private final SerializableFunction tableRefFunction; @@ -2716,15 +2802,16 @@ private static class TagWithUniqueIdsAndTable private transient String randomUUID; private transient long sequenceNo = 0L; - TagWithUniqueIdsAndTable(BigQueryOptions options, TableReference table, + TagWithUniqueIdsAndTable(BigQueryOptions options, + ValueProvider table, SerializableFunction tableRefFunction) { checkArgument(table == null ^ tableRefFunction == null, "Exactly one of table or tableRefFunction should be set"); if (table != null) { - if (table.getProjectId() == null) { - table.setProjectId(options.as(BigQueryOptions.class).getProject()); + if (table.isAccessible() && table.get().getProjectId() == null) { + table.get().setProjectId(options.as(BigQueryOptions.class).getProject()); } - this.tableSpec = toTableSpec(table); + this.tableSpec = NestedValueProvider.of(table, new TableRefToTableSpec()); } else { tableSpec = null; } @@ -2763,7 +2850,7 @@ public void populateDisplayData(DisplayData.Builder builder) { private String tableSpecFromWindow(BigQueryOptions options, BoundedWindow window) { if (tableSpec != null) { - return tableSpec; + return tableSpec.get(); } else { TableReference table = tableRefFunction.apply(window); if (table.getProjectId() == null) { @@ -2781,15 +2868,15 @@ private String tableSpecFromWindow(BigQueryOptions options, BoundedWindow window * it leverages BigQuery best effort de-dup mechanism. */ private static class StreamWithDeDup extends PTransform, PDone> { - private final transient TableReference tableReference; + private final transient ValueProvider tableReference; private final SerializableFunction tableRefFunction; - private final transient TableSchema tableSchema; + private final transient ValueProvider tableSchema; private final BigQueryServices bqServices; /** Constructor. */ - StreamWithDeDup(TableReference tableReference, + StreamWithDeDup(ValueProvider tableReference, SerializableFunction tableRefFunction, - TableSchema tableSchema, + ValueProvider tableSchema, BigQueryServices bqServices) { this.tableReference = tableReference; this.tableRefFunction = tableRefFunction; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 25caf63f18ca..54ec2bb9b4cc 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -95,6 +95,7 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryQuerySource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.BigQueryTableSource; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.JsonSchemaToTableSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.PassThroughThenCleanup; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.PassThroughThenCleanup.CleanupOperation; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Status; @@ -111,6 +112,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.NeedsRunner; @@ -643,9 +646,9 @@ private void checkWriteObjectWithValidate( BigQueryIO.Write.Bound bound, String project, String dataset, String table, TableSchema schema, CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate) { - assertEquals(project, bound.getTable().getProjectId()); - assertEquals(dataset, bound.getTable().getDatasetId()); - assertEquals(table, bound.getTable().getTableId()); + assertEquals(project, bound.getTable().get().getProjectId()); + assertEquals(dataset, bound.getTable().get().getDatasetId()); + assertEquals(table, bound.getTable().get().getTableId()); assertEquals(schema, bound.getSchema()); assertEquals(createDisposition, bound.createDisposition); assertEquals(writeDisposition, bound.writeDisposition); @@ -1845,8 +1848,8 @@ public void testWriteTables() throws Exception { fakeBqServices, jobIdToken, tempFilePrefix, - jsonTable, - jsonSchema, + StaticValueProvider.of(jsonTable), + StaticValueProvider.of(jsonSchema), WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED); @@ -1920,7 +1923,7 @@ public void testWriteRename() throws Exception { WriteRename writeRename = new WriteRename( fakeBqServices, jobIdToken, - jsonTable, + StaticValueProvider.of(jsonTable), WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, tempTablesView); @@ -1961,6 +1964,51 @@ public void testRemoveTemporaryTables() throws Exception { logged.verifyNotLogged("Failed to delete the table " + toJsonString(tableRefs.get(2))); } + /** Test options. **/ + public interface RuntimeTestOptions extends PipelineOptions { + ValueProvider getInputTable(); + void setInputTable(ValueProvider value); + + ValueProvider getInputQuery(); + void setInputQuery(ValueProvider value); + + ValueProvider getOutputTable(); + void setOutputTable(ValueProvider value); + + ValueProvider getOutputSchema(); + void setOutputSchema(ValueProvider value); + } + + @Test + public void testRuntimeOptionsNotCalledInApplyInputTable() { + RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); + BigQueryOptions bqOptions = options.as(BigQueryOptions.class); + bqOptions.setTempLocation("gs://testbucket/testdir"); + Pipeline pipeline = TestPipeline.create(options); + pipeline + .apply(BigQueryIO.Read.from(options.getInputTable()).withoutValidation()) + .apply(BigQueryIO.Write + .to(options.getOutputTable()) + .withSchema(NestedValueProvider.of( + options.getOutputSchema(), new JsonSchemaToTableSchema())) + .withoutValidation()); + } + + @Test + public void testRuntimeOptionsNotCalledInApplyInputQuery() { + RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); + BigQueryOptions bqOptions = options.as(BigQueryOptions.class); + bqOptions.setTempLocation("gs://testbucket/testdir"); + Pipeline pipeline = TestPipeline.create(options); + pipeline + .apply(BigQueryIO.Read.fromQuery(options.getInputQuery()).withoutValidation()) + .apply(BigQueryIO.Write + .to(options.getOutputTable()) + .withSchema(NestedValueProvider.of( + options.getOutputSchema(), new JsonSchemaToTableSchema())) + .withoutValidation()); + } + private static void testNumFiles(File tempDir, int expectedNumFiles) { assertEquals(expectedNumFiles, tempDir.listFiles(new FileFilter() { @Override From 6b14ce538c52d26c3b6a5db3b8b1f603216b21d8 Mon Sep 17 00:00:00 2001 From: Sela Date: Mon, 12 Dec 2016 12:37:32 +0200 Subject: [PATCH 151/279] [BEAM-1133] Add maxNumRecords per micro-batch for Spark runner options. --- .../runners/spark/SparkPipelineOptions.java | 5 +++++ .../beam/runners/spark/io/SourceDStream.java | 21 ++++++++++++++----- .../spark/io/SparkUnboundedSource.java | 17 +++++++++------ 3 files changed, 32 insertions(+), 11 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java index 3f8b3791f151..a2cd8879c154 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java @@ -54,6 +54,11 @@ public interface SparkPipelineOptions Long getMinReadTimeMillis(); void setMinReadTimeMillis(Long minReadTimeMillis); + @Description("Max records per micro-batch. For streaming sources only.") + @Default.Long(-1) + Long getMaxRecordsPerBatch(); + void setMaxRecordsPerBatch(Long maxRecordsPerBatch); + @Description("A value between 0-1 to describe the percentage of a micro-batch dedicated " + "to reading from UnboundedSource.") @Default.Double(0.1) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java index 84b247b265da..8a0763b7052b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java @@ -53,7 +53,7 @@ * {@link SparkPipelineOptions#getMinReadTimeMillis()}. * Records bound is controlled by the {@link RateController} mechanism. */ -public class SourceDStream +class SourceDStream extends InputDStream, CheckpointMarkT>> { private static final Logger LOG = LoggerFactory.getLogger(SourceDStream.class); @@ -64,10 +64,16 @@ public class SourceDStream unboundedSource, + SparkRuntimeContext runtimeContext) { - public SourceDStream(StreamingContext ssc, - UnboundedSource unboundedSource, - SparkRuntimeContext runtimeContext) { super(ssc, JavaSparkContext$.MODULE$., CheckpointMarkT>>fakeClassTag()); this.unboundedSource = unboundedSource; this.runtimeContext = runtimeContext; @@ -80,10 +86,15 @@ public SourceDStream(StreamingContext ssc, checkArgument(this.initialParallelism > 0, "Number of partitions must be greater than zero."); } + public void setMaxRecordsPerBatch(long maxRecordsPerBatch) { + boundMaxRecords = maxRecordsPerBatch; + } + @Override public scala.Option, CheckpointMarkT>>> compute(Time validTime) { + long maxNumRecords = boundMaxRecords != null ? boundMaxRecords : rateControlledMaxRecords(); MicrobatchSource microbatchSource = new MicrobatchSource<>( - unboundedSource, boundReadDuration, initialParallelism, rateControlledMaxRecords(), -1, + unboundedSource, boundReadDuration, initialParallelism, maxNumRecords, -1, id()); RDD, CheckpointMarkT>> rdd = new SourceRDD.Unbounded<>( ssc().sc(), runtimeContext, microbatchSource); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java index b12098d79eab..394b02373436 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java @@ -61,19 +61,25 @@ public class SparkUnboundedSource { JavaDStream> read(JavaStreamingContext jssc, SparkRuntimeContext rc, UnboundedSource source) { + SparkPipelineOptions options = rc.getPipelineOptions().as(SparkPipelineOptions.class); + Long maxRecordsPerBatch = options.getMaxRecordsPerBatch(); + SourceDStream sourceDStream = new SourceDStream<>(jssc.ssc(), source, rc); + // if max records per batch was set by the user. + if (maxRecordsPerBatch > 0) { + sourceDStream.setMaxRecordsPerBatch(maxRecordsPerBatch); + } JavaPairInputDStream, CheckpointMarkT> inputDStream = - JavaPairInputDStream$.MODULE$.fromInputDStream(new SourceDStream<>(jssc.ssc(), source, rc), + JavaPairInputDStream$.MODULE$.fromInputDStream(sourceDStream, JavaSparkContext$.MODULE$.>fakeClassTag(), JavaSparkContext$.MODULE$.fakeClassTag()); // call mapWithState to read from a checkpointable sources. - //TODO: consider broadcasting the rc instead of re-sending every batch. JavaMapWithStateDStream, CheckpointMarkT, byte[], Iterator>> mapWithStateDStream = inputDStream.mapWithState( StateSpec.function(StateSpecFunctions.mapSourceFunction(rc))); // set checkpoint duration for read stream, if set. - checkpointStream(mapWithStateDStream, rc); + checkpointStream(mapWithStateDStream, options); // flatmap and report read elements. Use the inputDStream's id to tie between the reported // info and the inputDStream it originated from. int id = inputDStream.inputDStream().id(); @@ -97,9 +103,8 @@ private static String getSourceName(Source source, int id) { } private static void checkpointStream(JavaDStream dStream, - SparkRuntimeContext rc) { - long checkpointDurationMillis = rc.getPipelineOptions().as(SparkPipelineOptions.class) - .getCheckpointDurationMillis(); + SparkPipelineOptions options) { + long checkpointDurationMillis = options.getCheckpointDurationMillis(); if (checkpointDurationMillis > 0) { dStream.checkpoint(new Duration(checkpointDurationMillis)); } From 74b0befb33ad3dd55181444255395ced202d3188 Mon Sep 17 00:00:00 2001 From: Sela Date: Mon, 12 Dec 2016 12:41:34 +0200 Subject: [PATCH 152/279] [BEAM-1130] SparkRunner ResumeFromCheckpointStreamingTest Failing. --- .../streaming/KafkaStreamingTest.java | 34 ++++++++++++------- .../ResumeFromCheckpointStreamingTest.java | 17 ++++++---- 2 files changed, 31 insertions(+), 20 deletions(-) diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java index d55ed39dbdd7..6be92d050b6f 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/KafkaStreamingTest.java @@ -76,10 +76,15 @@ public static void init() throws IOException { @Test public void testEarliest2Topics() throws Exception { + Duration batchIntervalDuration = Duration.standardSeconds(5); SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir); - // It seems that the consumer's first "position" lookup (in unit test) takes +200 msec, - // so to be on the safe side we'll set to 750 msec. - options.setMinReadTimeMillis(750L); + // provide a generous enough batch-interval to have everything fit in one micro-batch. + options.setBatchIntervalMillis(batchIntervalDuration.getMillis()); + // provide a very generous read time bound, we rely on num records bound here. + options.setMinReadTimeMillis(batchIntervalDuration.minus(1).getMillis()); + // bound the read on the number of messages - 2 topics of 4 messages each. + options.setMaxRecordsPerBatch(8L); + //--- setup // two topics. final String topic1 = "topic1"; @@ -90,8 +95,6 @@ public void testEarliest2Topics() throws Exception { ); // expected. final String[] expected = {"k1,v1", "k2,v2", "k3,v3", "k4,v4"}; - // batch and window duration. - final Duration batchAndWindowDuration = Duration.standardSeconds(1); // write to both topics ahead. produce(topic1, messages); @@ -114,17 +117,27 @@ public void testEarliest2Topics() throws Exception { PCollection deduped = p.apply(read.withoutMetadata()).setCoder( KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) - .apply(Window.>into(FixedWindows.of(batchAndWindowDuration))) + .apply(Window.>into(FixedWindows.of(batchIntervalDuration))) .apply(ParDo.of(new FormatKVFn())) .apply(Distinct.create()); - PAssertStreaming.runAndAssertContents(p, deduped, expected, Duration.standardSeconds(1L)); + // graceful shutdown will make sure first batch (at least) will finish. + Duration timeout = Duration.standardSeconds(1L); + PAssertStreaming.runAndAssertContents(p, deduped, expected, timeout); } @Test public void testLatest() throws Exception { + Duration batchIntervalDuration = Duration.standardSeconds(5); SparkContextOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir).as(SparkContextOptions.class); + // provide a generous enough batch-interval to have everything fit in one micro-batch. + options.setBatchIntervalMillis(batchIntervalDuration.getMillis()); + // provide a very generous read time bound, we rely on num records bound here. + options.setMinReadTimeMillis(batchIntervalDuration.minus(1).getMillis()); + // bound the read on the number of messages - 1 topics of 4 messages. + options.setMaxRecordsPerBatch(4L); + //--- setup final String topic = "topic"; // messages. @@ -133,16 +146,11 @@ public void testLatest() throws Exception { ); // expected. final String[] expected = {"k1,v1", "k2,v2", "k3,v3", "k4,v4"}; - // batch and window duration. - final Duration batchAndWindowDuration = Duration.standardSeconds(1); // write once first batch completes, this will guarantee latest-like behaviour. options.setListeners(Collections.singletonList( KafkaWriteOnBatchCompleted.once(messages, Collections.singletonList(topic), EMBEDDED_KAFKA_CLUSTER.getProps(), EMBEDDED_KAFKA_CLUSTER.getBrokerList()))); - // It seems that the consumer's first "position" lookup (in unit test) takes +200 msec, - // so to be on the safe side we'll set to 750 msec. - options.setMinReadTimeMillis(750L); //------- test: read and format. Pipeline p = Pipeline.create(options); @@ -161,7 +169,7 @@ public void testLatest() throws Exception { PCollection formatted = p.apply(read.withoutMetadata()).setCoder( KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) - .apply(Window.>into(FixedWindows.of(batchAndWindowDuration))) + .apply(Window.>into(FixedWindows.of(batchIntervalDuration))) .apply(ParDo.of(new FormatKVFn())); // run for more than 1 batch interval, so that reading of latest is attempted in the diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java index 945ee765610e..2718b5f7c9f2 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java @@ -112,10 +112,14 @@ private static void produce() { @Test public void testRun() throws Exception { + Duration batchIntervalDuration = Duration.standardSeconds(5); SparkPipelineOptions options = commonOptions.withTmpCheckpointDir(checkpointParentDir); - // It seems that the consumer's first "position" lookup (in unit test) takes +200 msec, - // so to be on the safe side we'll set to 750 msec. - options.setMinReadTimeMillis(750L); + // provide a generous enough batch-interval to have everything fit in one micro-batch. + options.setBatchIntervalMillis(batchIntervalDuration.getMillis()); + // provide a very generous read time bound, we rely on num records bound here. + options.setMinReadTimeMillis(batchIntervalDuration.minus(1).getMillis()); + // bound the read on the number of messages - 1 topic of 4 messages. + options.setMaxRecordsPerBatch(4L); // checkpoint after first (and only) interval. options.setCheckpointDurationMillis(options.getBatchIntervalMillis()); @@ -164,10 +168,9 @@ private static SparkPipelineResult run(SparkPipelineOptions options) { .apply(Window.>into(FixedWindows.of(windowDuration))) .apply(ParDo.of(new FormatAsText())); - // requires a graceful stop so that checkpointing of the first run would finish successfully - // before stopping and attempting to resume. - return PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED, - Duration.standardSeconds(1L)); + // graceful shutdown will make sure first batch (at least) will finish. + Duration timeout = Duration.standardSeconds(1L); + return PAssertStreaming.runAndAssertContents(p, formattedKV, EXPECTED, timeout); } @AfterClass From cfb71e741d12544112f72a2a950204a07672bc25 Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 9 Dec 2016 14:05:30 -0800 Subject: [PATCH 153/279] Enable and fix DirectRunnerTest case missing @Test Improve MetricMatchers mismatch description --- .../beam/runners/direct/DirectRunnerTest.java | 7 ++- .../beam/sdk/metrics/MetricMatchers.java | 45 ++++++++++++++++--- 2 files changed, 45 insertions(+), 7 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 3c860b152d04..eb0f344dabe5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -467,6 +467,7 @@ public Long decode(InputStream inStream, Context context) throws IOException { } } + @Test public void testMetrics() throws Exception { Pipeline pipeline = getPipeline(); pipeline @@ -485,10 +486,12 @@ public void processElement(ProcessContext c) { MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder() .addNameFilter(MetricNameFilter.inNamespace(DirectRunnerTest.class)) .build()); + + final String stepName = "MyStep/AnonymousParDo/AnonymousParMultiDo"; assertThat(metrics.counters(), contains( - metricResult(DirectRunnerTest.class.getName(), "count", "MyStep", 3L, 3L))); + metricResult(DirectRunnerTest.class.getName(), "count", stepName, 3L, 3L))); assertThat(metrics.distributions(), contains( - metricResult(DirectRunnerTest.class.getName(), "input", "MyStep", + metricResult(DirectRunnerTest.class.getName(), "input", stepName, DistributionResult.create(26L, 3L, 5L, 13L), DistributionResult.create(26L, 3L, 5L, 13L)))); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java index bdcb94f5eadc..6cd4c5245eca 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java @@ -72,15 +72,15 @@ public void describeTo(Description description) { public static Matcher> metricResult( final String namespace, final String name, final String step, - final T logical, final T physical) { + final T committed, final T attempted) { return new TypeSafeMatcher>() { @Override protected boolean matchesSafely(MetricResult item) { return Objects.equals(namespace, item.name().namespace()) && Objects.equals(name, item.name().name()) && Objects.equals(step, item.step()) - && Objects.equals(logical, item.committed()) - && Objects.equals(physical, item.attempted()); + && Objects.equals(committed, item.committed()) + && Objects.equals(attempted, item.attempted()); } @Override @@ -89,10 +89,45 @@ public void describeTo(Description description) { .appendText("MetricResult{inNamespace=").appendValue(namespace) .appendText(", name=").appendValue(name) .appendText(", step=").appendValue(step) - .appendText(", logical=").appendValue(logical) - .appendText(", physical=").appendValue(physical) + .appendText(", committed=").appendValue(committed) + .appendText(", attempted=").appendValue(attempted) .appendText("}"); } + + @Override + protected void describeMismatchSafely(MetricResult item, Description mismatchDescription) { + mismatchDescription.appendText("MetricResult{"); + if (!Objects.equals(namespace, item.name().namespace())) { + mismatchDescription + .appendText("inNamespace: ").appendValue(namespace) + .appendText(" != ").appendValue(item.name().namespace()); + } + + if (!Objects.equals(name, item.name().name())) { + mismatchDescription + .appendText("name: ").appendValue(name) + .appendText(" != ").appendValue(item.name().name()); + } + + if (!Objects.equals(step, item.step())) { + mismatchDescription + .appendText("step: ").appendValue(step) + .appendText(" != ").appendValue(item.step()); + } + + if (!Objects.equals(committed, item.committed())) { + mismatchDescription + .appendText("committed: ").appendValue(committed) + .appendText(" != ").appendValue(item.committed()); + } + + if (!Objects.equals(attempted, item.attempted())) { + mismatchDescription + .appendText("attempted: ").appendValue(attempted) + .appendText(" != ").appendValue(item.attempted()); + } + mismatchDescription.appendText("}"); + } }; } From 8f52b5e1214061a9cbd2bd2c44f68c22102d9737 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 9 Dec 2016 15:34:25 -0800 Subject: [PATCH 154/279] Migrate AppliedPTransform to use AutoValue --- .../sdk/transforms/AppliedPTransform.java | 75 ++++--------------- 1 file changed, 15 insertions(+), 60 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java index d80c116c175c..77de54a1bd57 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java @@ -17,8 +17,7 @@ */ package org.apache.beam.sdk.transforms; -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; +import com.google.auto.value.AutoValue; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; @@ -32,70 +31,26 @@ * @param transform output type * @param transform type */ -public class AppliedPTransform +@AutoValue +public abstract class AppliedPTransform > { - private final String fullName; - private final InputT input; - private final OutputT output; - private final TransformT transform; - - private AppliedPTransform(String fullName, InputT input, OutputT output, TransformT transform) { - this.input = input; - this.output = output; - this.transform = transform; - this.fullName = fullName; + public static < + InputT extends PInput, + OutputT extends POutput, + TransformT extends PTransform> + AppliedPTransform of( + String fullName, InputT input, OutputT output, TransformT transform) { + return new AutoValue_AppliedPTransform( + fullName, input, output, transform); } - public static > - AppliedPTransform of( - String fullName, InputT input, OutputT output, TransformT transform) { - return new AppliedPTransform(fullName, input, output, transform); - } + public abstract String getFullName(); - public String getFullName() { - return fullName; - } + public abstract InputT getInput(); - public InputT getInput() { - return input; - } + public abstract OutputT getOutput(); - public OutputT getOutput() { - return output; - } - - public TransformT getTransform() { - return transform; - } - - @Override - public int hashCode() { - return Objects.hashCode(getFullName(), getInput(), getOutput(), getTransform()); - } - - @Override - public boolean equals(Object other) { - if (other instanceof AppliedPTransform) { - AppliedPTransform that = (AppliedPTransform) other; - return Objects.equal(this.getFullName(), that.getFullName()) - && Objects.equal(this.getInput(), that.getInput()) - && Objects.equal(this.getOutput(), that.getOutput()) - && Objects.equal(this.getTransform(), that.getTransform()); - } else { - return false; - } - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("fullName", getFullName()) - .add("input", getInput()) - .add("output", getOutput()) - .add("transform", getTransform()) - .toString(); - } + public abstract TransformT getTransform(); } From 8ef74a744327c40fbb05030fd7657db8a865cb94 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 9 Dec 2016 15:52:15 -0800 Subject: [PATCH 155/279] Stop expanding PValues in DirectRunner visitors A PValue always expands to itself, and these calls are unneccessary. --- .../beam/runners/direct/DirectGraphVisitor.java | 14 ++++++-------- .../runners/direct/KeyedPValueTrackingVisitor.java | 2 +- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index 4f38bce1e9d9..0283d0352240 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -99,14 +99,12 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { if (!producers.containsKey(value)) { producers.put(value, appliedTransform); } - for (PValue expandedValue : value.expand()) { - if (expandedValue instanceof PCollectionView) { - views.add((PCollectionView) expandedValue); - } - if (!producers.containsKey(expandedValue)) { - producers.put(value, appliedTransform); - } - } + if (value instanceof PCollectionView) { + views.add((PCollectionView) value); + } + if (!producers.containsKey(value)) { + producers.put(value, appliedTransform); + } } private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 4161f9e25def..7f85169478a1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -84,7 +84,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) {} @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { if (producesKeyedOutputs.contains(producer.getTransform().getClass())) { - keyedValues.addAll(value.expand()); + keyedValues.add(value); } } From 9678b1cc6c799767e48ebc4d9071db099b4d135d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 3 Nov 2016 20:44:45 -0700 Subject: [PATCH 156/279] Add IntervalWindow coder to the standard registry --- .../src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java index 9110de058af6..65f4209acfb1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode; import org.apache.beam.sdk.coders.protobuf.ProtoCoder; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; @@ -107,6 +108,7 @@ public void registerStandardCoders() { registerCoder(TimestampedValue.class, TimestampedValue.TimestampedValueCoder.class); registerCoder(Void.class, VoidCoder.class); registerCoder(byte[].class, ByteArrayCoder.class); + registerCoder(IntervalWindow.class, IntervalWindow.getCoder()); } /** From db41940f977bf3315ea7e5460d188d8f9b4fa119 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 5 Dec 2016 14:32:12 -0800 Subject: [PATCH 157/279] Factor out ShardedFile from FileChecksumMatcher --- .../beam/sdk/testing/FileChecksumMatcher.java | 114 ++------- .../beam/sdk/util/ExplicitShardedFile.java | 120 ++++++++++ .../beam/sdk/util/NumberedShardedFile.java | 220 ++++++++++++++++++ .../org/apache/beam/sdk/util/ShardedFile.java | 42 ++++ .../sdk/testing/FileChecksumMatcherTest.java | 77 ------ .../sdk/util/NumberedShardedFileTest.java | 181 ++++++++++++++ 6 files changed, 581 insertions(+), 173 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java index 4b249fed38c5..82a6b7117636 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -21,31 +21,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; -import com.google.common.collect.Lists; import com.google.common.hash.HashCode; import com.google.common.hash.Hashing; -import com.google.common.io.CharStreams; -import java.io.IOException; -import java.io.Reader; -import java.nio.channels.Channels; import java.nio.charset.StandardCharsets; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.ArrayList; -import java.util.Collection; import java.util.List; -import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.annotation.Nonnull; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.NumberedShardedFile; +import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; @@ -83,9 +71,8 @@ public class FileChecksumMatcher extends TypeSafeMatcher Pattern.compile("(?x) \\S* (? \\d+) -of- (? \\d+)"); private final String expectedChecksum; - private final String filePath; - private final Pattern shardTemplate; private String actualChecksum; + private final ShardedFile shardedFile; /** * Constructor that uses default shard template. @@ -98,7 +85,7 @@ public FileChecksumMatcher(String checksum, String filePath) { } /** - * Constructor. + * Constructor using a custom shard template. * * @param checksum expected checksum string used to verify file content. * @param filePath path of files that's to be verified. @@ -121,8 +108,17 @@ public FileChecksumMatcher(String checksum, String filePath, Pattern shardTempla DEFAULT_SHARD_TEMPLATE); this.expectedChecksum = checksum; - this.filePath = filePath; - this.shardTemplate = shardTemplate; + this.shardedFile = new NumberedShardedFile(filePath, shardTemplate); + } + + /** + * Constructor using an entirely custom {@link ShardedFile} implementation. + * + *

      For internal use only. + */ + public FileChecksumMatcher(String expectedChecksum, ShardedFile shardedFile) { + this.expectedChecksum = expectedChecksum; + this.shardedFile = shardedFile; } @Override @@ -130,9 +126,10 @@ public boolean matchesSafely(PipelineResult pipelineResult) { // Load output data List outputs; try { - outputs = readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff()); + outputs = shardedFile.readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff()); } catch (Exception e) { - throw new RuntimeException(String.format("Failed to read from: %s", filePath), e); + throw new RuntimeException( + String.format("Failed to read from: %s", shardedFile), e); } // Verify outputs. Checksum is computed using SHA-1 algorithm @@ -142,81 +139,6 @@ public boolean matchesSafely(PipelineResult pipelineResult) { return actualChecksum.equals(expectedChecksum); } - @VisibleForTesting - List readFilesWithRetries(Sleeper sleeper, BackOff backOff) - throws IOException, InterruptedException { - IOChannelFactory factory = IOChannelUtils.getFactory(filePath); - IOException lastException = null; - - do { - try { - // Match inputPath which may contains glob - Collection files = factory.match(filePath); - LOG.debug("Found {} file(s) by matching the path: {}", files.size(), filePath); - - if (files.isEmpty() || !checkTotalNumOfFiles(files)) { - continue; - } - - // Read data from file paths - return readLines(files, factory); - } catch (IOException e) { - // Ignore and retry - lastException = e; - LOG.warn("Error in file reading. Ignore and retry."); - } - } while(BackOffUtils.next(sleeper, backOff)); - // Failed after max retries - throw new IOException( - String.format("Unable to read file(s) after retrying %d times", MAX_READ_RETRIES), - lastException); - } - - @VisibleForTesting - List readLines(Collection files, IOChannelFactory factory) throws IOException { - List allLines = Lists.newArrayList(); - int i = 1; - for (String file : files) { - try (Reader reader = - Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { - List lines = CharStreams.readLines(reader); - allLines.addAll(lines); - LOG.debug( - "[{} of {}] Read {} lines from file: {}", i, files.size(), lines.size(), file); - } - i++; - } - return allLines; - } - - /** - * Check if total number of files is correct by comparing with the number that - * is parsed from shard name using a name template. If no template is specified, - * "SSSS-of-NNNN" will be used as default, and "NNNN" will be the expected total - * number of files. - * - * @return {@code true} if at least one shard name matches template and total number - * of given files equals the number that is parsed from shard name. - */ - @VisibleForTesting - boolean checkTotalNumOfFiles(Collection files) { - for (String filePath : files) { - Path fileName = Paths.get(filePath).getFileName(); - if (fileName == null) { - // this path has zero elements - continue; - } - Matcher matcher = shardTemplate.matcher(fileName.toString()); - if (!matcher.matches()) { - // shard name doesn't match the pattern, check with the next shard - continue; - } - // once match, extract total number of shards and compare to file list - return files.size() == Integer.parseInt(matcher.group("numshards")); - } - return false; - } - private String computeHash(@Nonnull List strs) { if (strs.isEmpty()) { return Hashing.sha1().hashString("", StandardCharsets.UTF_8).toString(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java new file mode 100644 index 000000000000..5f5bf1ffaf72 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.io.CharStreams; +import java.io.IOException; +import java.io.Reader; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A sharded file where the file names are simply provided. */ +public class ExplicitShardedFile implements ShardedFile { + + private static final Logger LOG = LoggerFactory.getLogger(ExplicitShardedFile.class); + + private static final int MAX_READ_RETRIES = 4; + private static final Duration DEFAULT_SLEEP_DURATION = Duration.standardSeconds(10L); + static final FluentBackoff BACK_OFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(DEFAULT_SLEEP_DURATION) + .withMaxRetries(MAX_READ_RETRIES); + + private final Collection files; + + /** Constructs an {@link ExplicitShardedFile} for the given files. */ + public ExplicitShardedFile(Collection files) { + this.files = files; + } + + @Override + public List readFilesWithRetries(Sleeper sleeper, BackOff backOff) + throws IOException, InterruptedException { + if (files.isEmpty()) { + return Collections.emptyList(); + } + + IOChannelFactory factory = IOChannelUtils.getFactory(Iterables.get(files, 0)); + IOException lastException = null; + + do { + try { + // Read data from file paths + return readLines(files, factory); + } catch (IOException e) { + // Ignore and retry + lastException = e; + LOG.warn("Error in file reading. Ignore and retry."); + } + } while (BackOffUtils.next(sleeper, backOff)); + // Failed after max retries + throw new IOException( + String.format("Unable to read file(s) after retrying %d times", MAX_READ_RETRIES), + lastException); + } + + /** + * Discovers all shards of this file using the provided {@link Sleeper} and {@link BackOff}. + * + *

      Because of eventual consistency, reads may discover no files or fewer files than the shard + * template implies. In this case, the read is considered to have failed. + */ + public List readFilesWithRetries() throws IOException, InterruptedException { + return readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff()); + } + + @Override + public String toString() { + return String.format("explicit sharded file (%s)", Joiner.on(", ").join(files)); + } + + /** + * Reads all the lines of all the files. + * + *

      Not suitable for use except in testing of small data, since the data size may be far more + * than can be reasonably processed serially, in-memory, by a single thread. + */ + @VisibleForTesting + List readLines(Collection files, IOChannelFactory factory) throws IOException { + List allLines = Lists.newArrayList(); + int i = 1; + for (String file : files) { + try (Reader reader = Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { + List lines = CharStreams.readLines(reader); + allLines.addAll(lines); + LOG.debug("[{} of {}] Read {} lines from file: {}", i, files.size(), lines.size(), file); + } + i++; + } + return allLines; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java new file mode 100644 index 000000000000..f9f2d6d5dc9b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hashing; +import com.google.common.io.CharStreams; +import java.io.IOException; +import java.io.Reader; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nonnull; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility methods for working with sharded files. For internal use only; many parameters + * are just hardcoded to allow existing uses to work OK. + */ +public class NumberedShardedFile implements ShardedFile { + + private static final Logger LOG = LoggerFactory.getLogger(NumberedShardedFile.class); + + static final int MAX_READ_RETRIES = 4; + static final Duration DEFAULT_SLEEP_DURATION = Duration.standardSeconds(10L); + static final FluentBackoff BACK_OFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(DEFAULT_SLEEP_DURATION) + .withMaxRetries(MAX_READ_RETRIES); + + private static final Pattern DEFAULT_SHARD_TEMPLATE = + Pattern.compile("(?x) \\S* (? \\d+) -of- (? \\d+)"); + + private final String filePath; + private final Pattern shardTemplate; + + /** + * Constructor that uses default shard template. + * + * @param filePath path or glob of files to include + */ + public NumberedShardedFile(String filePath) { + this(filePath, DEFAULT_SHARD_TEMPLATE); + } + + /** + * Constructor. + * + * @param filePath path or glob of files to include + * @param shardTemplate template of shard name to parse out the total number of shards + * which is used in I/O retry to avoid inconsistency of filesystem. + * Customized template should assign name "numshards" to capturing + * group - total shard number. + */ + public NumberedShardedFile(String filePath, Pattern shardTemplate) { + checkArgument( + !Strings.isNullOrEmpty(filePath), + "Expected valid file path, but received %s", filePath); + checkNotNull( + shardTemplate, + "Expected non-null shard pattern. " + + "Please call the other constructor to use default pattern: %s", + DEFAULT_SHARD_TEMPLATE); + + this.filePath = filePath; + this.shardTemplate = shardTemplate; + } + + public String getFilePath() { + return filePath; + } + + /** + * Discovers all shards of this file using the provided {@link Sleeper} and {@link BackOff}. + * + *

      Because of eventual consistency, reads may discover no files or fewer files than + * the shard template implies. In this case, the read is considered to have failed. + */ + @Override + public List readFilesWithRetries(Sleeper sleeper, BackOff backOff) + throws IOException, InterruptedException { + IOChannelFactory factory = IOChannelUtils.getFactory(filePath); + IOException lastException = null; + + do { + try { + // Match inputPath which may contains glob + Collection files = factory.match(filePath); + LOG.debug("Found {} file(s) by matching the path: {}", files.size(), filePath); + + if (files.isEmpty() || !checkTotalNumOfFiles(files)) { + continue; + } + + // Read data from file paths + return readLines(files, factory); + } catch (IOException e) { + // Ignore and retry + lastException = e; + LOG.warn("Error in file reading. Ignore and retry."); + } + } while(BackOffUtils.next(sleeper, backOff)); + // Failed after max retries + throw new IOException( + String.format("Unable to read file(s) after retrying %d times", MAX_READ_RETRIES), + lastException); + } + + /** + * Discovers all shards of this file using the provided {@link Sleeper} and {@link BackOff}. + * + *

      Because of eventual consistency, reads may discover no files or fewer files than + * the shard template implies. In this case, the read is considered to have failed. + */ + public List readFilesWithRetries() + throws IOException, InterruptedException { + return readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff()); + } + + @Override + public String toString() { + return String.format("%s with shard template '%s'", filePath, shardTemplate); + } + + /** + * Reads all the lines of all the files. + * + *

      Not suitable for use except in testing of small data, since the data size may be far more + * than can be reasonably processed serially, in-memory, by a single thread. + */ + @VisibleForTesting + List readLines(Collection files, IOChannelFactory factory) throws IOException { + List allLines = Lists.newArrayList(); + int i = 1; + for (String file : files) { + try (Reader reader = + Channels.newReader(factory.open(file), StandardCharsets.UTF_8.name())) { + List lines = CharStreams.readLines(reader); + allLines.addAll(lines); + LOG.debug( + "[{} of {}] Read {} lines from file: {}", i, files.size(), lines.size(), file); + } + i++; + } + return allLines; + } + + /** + * Check if total number of files is correct by comparing with the number that + * is parsed from shard name using a name template. If no template is specified, + * "SSSS-of-NNNN" will be used as default, and "NNNN" will be the expected total + * number of files. + * + * @return {@code true} if at least one shard name matches template and total number + * of given files equals the number that is parsed from shard name. + */ + @VisibleForTesting + boolean checkTotalNumOfFiles(Collection files) { + for (String filePath : files) { + Path fileName = Paths.get(filePath).getFileName(); + if (fileName == null) { + // this path has zero elements + continue; + } + Matcher matcher = shardTemplate.matcher(fileName.toString()); + if (!matcher.matches()) { + // shard name doesn't match the pattern, check with the next shard + continue; + } + // once match, extract total number of shards and compare to file list + return files.size() == Integer.parseInt(matcher.group("numshards")); + } + return false; + } + + private String computeHash(@Nonnull List strs) { + if (strs.isEmpty()) { + return Hashing.sha1().hashString("", StandardCharsets.UTF_8).toString(); + } + + List hashCodes = new ArrayList<>(); + for (String str : strs) { + hashCodes.add(Hashing.sha1().hashString(str, StandardCharsets.UTF_8)); + } + return Hashing.combineUnordered(hashCodes).toString(); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java new file mode 100644 index 000000000000..ec9ed641976c --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.Sleeper; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import org.apache.beam.sdk.testing.SerializableMatcher; + +/** + * Bare-bones class for using sharded files. + * + *

      For internal use only; used only in SDK tests. Must be {@link Serializable} so it can be + * shipped as a {@link SerializableMatcher}. + */ +public interface ShardedFile extends Serializable { + + /** + * Reads the lines from all shards of this file using the provided {@link Sleeper} and {@link + * BackOff}. + */ + List readFilesWithRetries(Sleeper sleeper, BackOff backOff) + throws IOException, InterruptedException; +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java index 0dc307d528c2..5438479273be 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -19,10 +19,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyCollection; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.spy; import com.google.api.client.util.BackOff; import com.google.common.io.Files; @@ -30,9 +26,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; - import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; import org.junit.Rule; import org.junit.Test; @@ -76,13 +70,6 @@ public void testPreconditionChecksumIsEmpty() throws IOException { new FileChecksumMatcher("", tmpPath); } - @Test - public void testPreconditionFilePathIsNull() { - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(containsString("Expected valid file path, but received")); - new FileChecksumMatcher("checksumString", null); - } - @Test public void testPreconditionFilePathIsEmpty() { thrown.expect(IllegalArgumentException.class); @@ -158,68 +145,4 @@ public void testMatcherThatUsesCustomizedTemplate() throws Exception { assertThat(pResult, matcher); } - - @Test - public void testReadWithRetriesFailsWhenTemplateIncorrect() throws Exception { - File tmpFile = tmpFolder.newFile(); - Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); - - FileChecksumMatcher matcher = new FileChecksumMatcher( - "mock-checksum", - IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"), - Pattern.compile("incorrect-template")); - - thrown.expect(IOException.class); - thrown.expectMessage( - containsString( - "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); - matcher.readFilesWithRetries(fastClock, backOff); - } - - @Test - public void testReadWithRetriesFailsSinceFilesystemError() throws Exception { - File tmpFile = tmpFolder.newFile(); - Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); - - FileChecksumMatcher matcher = - spy(new FileChecksumMatcher( - "mock-checksum", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"))); - doThrow(IOException.class) - .when(matcher).readLines(anyCollection(), any(IOChannelFactory.class)); - - thrown.expect(IOException.class); - thrown.expectMessage( - containsString( - "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); - matcher.readFilesWithRetries(fastClock, backOff); - } - - @Test - public void testReadWithRetriesFailsWhenOutputDirEmpty() throws Exception { - FileChecksumMatcher matcher = - new FileChecksumMatcher( - "mock-checksum", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); - - thrown.expect(IOException.class); - thrown.expectMessage( - containsString( - "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); - matcher.readFilesWithRetries(fastClock, backOff); - } - - @Test - public void testReadWithRetriesFailsWhenRedundantFileLoaded() throws Exception { - tmpFolder.newFile("result-000-of-001"); - tmpFolder.newFile("tmp-result-000-of-001"); - - FileChecksumMatcher matcher = - new FileChecksumMatcher( - "mock-checksum", IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); - - thrown.expect(IOException.class); - thrown.expectMessage( - containsString( - "Unable to read file(s) after retrying " + FileChecksumMatcher.MAX_READ_RETRIES)); - matcher.readFilesWithRetries(fastClock, backOff); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java new file mode 100644 index 000000000000..475e459e7346 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyCollection; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; + +import com.google.api.client.util.BackOff; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.regex.Pattern; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.Mockito; + +/** Tests for {@link NumberedShardedFile}. */ +@RunWith(JUnit4.class) +public class NumberedShardedFileTest { + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule public ExpectedException thrown = ExpectedException.none(); + @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper(); + + @Mock private PipelineResult pResult = Mockito.mock(PipelineResult.class); + + private final BackOff backOff = NumberedShardedFile.BACK_OFF_FACTORY.backoff(); + + @Test + public void testPreconditionFilePathIsNull() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid file path, but received")); + new NumberedShardedFile(null); + } + + @Test + public void testPreconditionFilePathIsEmpty() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid file path, but received")); + new NumberedShardedFile(""); + } + + @Test + public void testReadMultipleShards() throws Exception { + String + contents1 = "To be or not to be, ", + contents2 = "it is not a question.", + contents3 = "should not be included"; + + File tmpFile1 = tmpFolder.newFile("result-000-of-002"); + File tmpFile2 = tmpFolder.newFile("result-001-of-002"); + File tmpFile3 = tmpFolder.newFile("tmp"); + Files.write(contents1, tmpFile1, StandardCharsets.UTF_8); + Files.write(contents2, tmpFile2, StandardCharsets.UTF_8); + Files.write(contents3, tmpFile3, StandardCharsets.UTF_8); + + NumberedShardedFile shardedFile = + new NumberedShardedFile(IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "result-*")); + + assertThat(shardedFile.readFilesWithRetries(), containsInAnyOrder(contents1, contents2)); + } + + @Test + public void testReadEmpty() throws Exception { + File emptyFile = tmpFolder.newFile("result-000-of-001"); + Files.write("", emptyFile, StandardCharsets.UTF_8); + NumberedShardedFile shardedFile = + new NumberedShardedFile(IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); + + assertThat(shardedFile.readFilesWithRetries(), empty()); + } + + @Test + public void testReadCustomTemplate() throws Exception { + String contents1 = "To be or not to be, ", contents2 = "it is not a question."; + + // Customized template: resultSSS-totalNNN + File tmpFile1 = tmpFolder.newFile("result0-total2"); + File tmpFile2 = tmpFolder.newFile("result1-total2"); + Files.write(contents1, tmpFile1, StandardCharsets.UTF_8); + Files.write(contents2, tmpFile2, StandardCharsets.UTF_8); + + Pattern customizedTemplate = + Pattern.compile("(?x) result (?\\d+) - total (?\\d+)"); + NumberedShardedFile shardedFile = + new NumberedShardedFile( + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"), customizedTemplate); + + assertThat(shardedFile.readFilesWithRetries(), containsInAnyOrder(contents1, contents2)); + } + + @Test + public void testReadWithRetriesFailsWhenTemplateIncorrect() throws Exception { + File tmpFile = tmpFolder.newFile(); + Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); + + NumberedShardedFile shardedFile = + new NumberedShardedFile( + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"), + Pattern.compile("incorrect-template")); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + NumberedShardedFile.MAX_READ_RETRIES)); + shardedFile.readFilesWithRetries(fastClock, backOff); + } + + @Test + public void testReadWithRetriesFailsSinceFilesystemError() throws Exception { + File tmpFile = tmpFolder.newFile(); + Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); + + NumberedShardedFile shardedFile = + spy(new NumberedShardedFile(IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*"))); + doThrow(IOException.class) + .when(shardedFile) + .readLines(anyCollection(), any(IOChannelFactory.class)); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + NumberedShardedFile.MAX_READ_RETRIES)); + shardedFile.readFilesWithRetries(fastClock, backOff); + } + + @Test + public void testReadWithRetriesFailsWhenOutputDirEmpty() throws Exception { + NumberedShardedFile shardedFile = + new NumberedShardedFile(IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + NumberedShardedFile.MAX_READ_RETRIES)); + shardedFile.readFilesWithRetries(fastClock, backOff); + } + + @Test + public void testReadWithRetriesFailsWhenRedundantFileLoaded() throws Exception { + tmpFolder.newFile("result-000-of-001"); + tmpFolder.newFile("tmp-result-000-of-001"); + + NumberedShardedFile shardedFile = + new NumberedShardedFile(IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); + + thrown.expect(IOException.class); + thrown.expectMessage( + containsString( + "Unable to read file(s) after retrying " + NumberedShardedFile.MAX_READ_RETRIES)); + shardedFile.readFilesWithRetries(fastClock, backOff); + } +} From 42595dcd29c248bd3572596c9bb8464d18acd19b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 3 Nov 2016 14:37:26 -0700 Subject: [PATCH 158/279] Revise WindowedWordCount for runner and execution mode portability --- .../beam/examples/WindowedWordCount.java | 177 +++++++++-------- .../common/WriteWindowedFilesDoFn.java | 77 ++++++++ .../beam/examples/WindowedWordCountIT.java | 182 +++++++++++++++--- 3 files changed, 326 insertions(+), 110 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/common/WriteWindowedFilesDoFn.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index 4e254bd0e56e..5c19454d244b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -17,26 +17,25 @@ */ package org.apache.beam.examples; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; -import org.apache.beam.examples.common.ExampleUtils; +import org.apache.beam.examples.common.WriteWindowedFilesDoFn; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -63,7 +62,8 @@ * 2. Adding timestamps to data * 3. Windowing * 4. Re-using PTransforms over windowed PCollections - * 5. Writing to BigQuery + * 5. Accessing the window of an element + * 6. Writing data to per-window text files * * *

      By default, the examples will run with the {@code DirectRunner}. @@ -74,25 +74,23 @@ * * See examples/java/README.md for instructions about how to configure different runners. * - *

      Optionally specify the input file path via: - * {@code --inputFile=gs://INPUT_PATH}, - * which defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt}. + *

      To execute this pipeline locally, specify a local output file (if using the + * {@code DirectRunner}) or output prefix on a supported distributed file system. + *

      {@code
      + *   --output=[YOUR_LOCAL_FILE | YOUR_OUTPUT_PREFIX]
      + * }
      * - *

      Specify an output BigQuery dataset and optionally, a table for the output. If you don't - * specify the table, one will be created for you using the job name. If you don't specify the - * dataset, a dataset called {@code beam_examples} must already exist in your project. - * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}. + *

      The input file defaults to a public data set containing the text of of King Lear, + * by William Shakespeare. You can override it and choose your own input with {@code --inputFile}. * *

      By default, the pipeline will do fixed windowing, on 1-minute windows. You can * change this interval by setting the {@code --windowSize} parameter, e.g. {@code --windowSize=10} * for 10-minute windows. * - *

      The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C) - * and then exits. + *

      The example will try to cancel the pipeline on the signal to terminate the process (CTRL-C). */ public class WindowedWordCount { - static final int WINDOW_SIZE = 1; // Default window duration in minutes - + static final int WINDOW_SIZE = 10; // Default window duration in minutes /** * Concept #2: A DoFn that sets the data element timestamp. This is a silly method, just for * this example, for the bounded data case. @@ -102,18 +100,22 @@ public class WindowedWordCount { * 2-hour period. */ static class AddTimestampFn extends DoFn { - private static final Duration RAND_RANGE = Duration.standardHours(2); + private static final Duration RAND_RANGE = Duration.standardHours(1); private final Instant minTimestamp; + private final Instant maxTimestamp; - AddTimestampFn() { - this.minTimestamp = new Instant(System.currentTimeMillis()); + AddTimestampFn(Instant minTimestamp, Instant maxTimestamp) { + this.minTimestamp = minTimestamp; + this.maxTimestamp = maxTimestamp; } @ProcessElement public void processElement(ProcessContext c) { - // Generate a timestamp that falls somewhere in the past two hours. - long randMillis = (long) (Math.random() * RAND_RANGE.getMillis()); - Instant randomTimestamp = minTimestamp.plus(randMillis); + Instant randomTimestamp = + new Instant( + ThreadLocalRandom.current() + .nextLong(minTimestamp.getMillis(), maxTimestamp.getMillis())); + /** * Concept #2: Set the data element with that timestamp. */ @@ -121,50 +123,29 @@ public void processElement(ProcessContext c) { } } - /** A DoFn that converts a Word and Count into a BigQuery table row. */ - static class FormatAsTableRowFn extends DoFn, TableRow> { - @ProcessElement - public void processElement(ProcessContext c) { - TableRow row = new TableRow() - .set("word", c.element().getKey()) - .set("count", c.element().getValue()) - // include a field for the window timestamp - .set("window_timestamp", c.timestamp().toString()); - c.output(row); + /** A {@link DefaultValueFactory} that returns the current system time. */ + public static class DefaultToCurrentSystemTime implements DefaultValueFactory { + @Override + public Long create(PipelineOptions options) { + return System.currentTimeMillis(); } } - /** - * Helper method that defines the BigQuery schema used for the output. - */ - private static TableSchema getSchema() { - List fields = new ArrayList<>(); - fields.add(new TableFieldSchema().setName("word").setType("STRING")); - fields.add(new TableFieldSchema().setName("count").setType("INTEGER")); - fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP")); - TableSchema schema = new TableSchema().setFields(fields); - return schema; - } - - /** - * Concept #5: We'll stream the results to a BigQuery table. The BigQuery output source is one - * that supports both bounded and unbounded data. This is a helper method that creates a - * TableReference from input options, to tell the pipeline where to write its BigQuery results. - */ - private static TableReference getTableReference(Options options) { - TableReference tableRef = new TableReference(); - tableRef.setProjectId(options.getProject()); - tableRef.setDatasetId(options.getBigQueryDataset()); - tableRef.setTableId(options.getBigQueryTable()); - return tableRef; + /** A {@link DefaultValueFactory} that returns the minimum timestamp plus one hour. */ + public static class DefaultToMinTimestampPlusOneHour implements DefaultValueFactory { + @Override + public Long create(PipelineOptions options) { + return options.as(Options.class).getMinTimestampMillis() + + Duration.standardHours(1).getMillis(); + } } /** - * Options supported by {@link WindowedWordCount}. + * Options for {@link WindowedWordCount}. * - *

      Inherits standard example configuration options, which allow specification of the BigQuery - * table, as well as the {@link WordCount.WordCountOptions} support for - * specification of the input file. + *

      Inherits standard example configuration options, which allow specification of the + * runner, as well as the {@link WordCount.WordCountOptions} support for + * specification of the input and output files. */ public interface Options extends WordCount.WordCountOptions, ExampleOptions, ExampleBigQueryTableOptions { @@ -172,14 +153,24 @@ public interface Options extends WordCount.WordCountOptions, @Default.Integer(WINDOW_SIZE) Integer getWindowSize(); void setWindowSize(Integer value); + + @Description("Minimum randomly assigned timestamp, in milliseconds-since-epoch") + @Default.InstanceFactory(DefaultToCurrentSystemTime.class) + Long getMinTimestampMillis(); + void setMinTimestampMillis(Long value); + + @Description("Maximum randomly assigned timestamp, in milliseconds-since-epoch") + @Default.InstanceFactory(DefaultToMinTimestampPlusOneHour.class) + Long getMaxTimestampMillis(); + void setMaxTimestampMillis(Long value); } public static void main(String[] args) throws IOException { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - options.setBigQuerySchema(getSchema()); - // ExampleUtils creates the necessary input sources to simplify execution of this Pipeline. - ExampleUtils exampleUtils = new ExampleUtils(options); - exampleUtils.setup(); + final String output = options.getOutput(); + final Duration windowSize = Duration.standardMinutes(options.getWindowSize()); + final Instant minTimestamp = new Instant(options.getMinTimestampMillis()); + final Instant maxTimestamp = new Instant(options.getMaxTimestampMillis()); Pipeline pipeline = Pipeline.create(options); @@ -192,7 +183,7 @@ public static void main(String[] args) throws IOException { .apply(TextIO.Read.from(options.getInputFile())) // Concept #2: Add an element timestamp, using an artificial time just to show windowing. // See AddTimestampFn for more detail on this. - .apply(ParDo.of(new AddTimestampFn())); + .apply(ParDo.of(new AddTimestampFn(minTimestamp, maxTimestamp))); /** * Concept #3: Window into fixed windows. The fixed window size for this example defaults to 1 @@ -200,9 +191,10 @@ public static void main(String[] args) throws IOException { * information on how fixed windows work, and for information on the other types of windowing * available (e.g., sliding windows). */ - PCollection windowedWords = input - .apply(Window.into( - FixedWindows.of(Duration.standardMinutes(options.getWindowSize())))); + PCollection windowedWords = + input.apply( + Window.into( + FixedWindows.of(Duration.standardMinutes(options.getWindowSize())))); /** * Concept #4: Re-use our existing CountWords transform that does not have knowledge of @@ -211,19 +203,40 @@ public static void main(String[] args) throws IOException { PCollection> wordCounts = windowedWords.apply(new WordCount.CountWords()); /** - * Concept #5: Format the results for a BigQuery table, then write to BigQuery. - * The BigQuery output source supports both bounded and unbounded data. + * Concept #5: Customize the output format using windowing information + * + *

      At this point, the data is organized by window. We're writing text files and and have no + * late data, so for simplicity we can use the window as the key and {@link GroupByKey} to get + * one output file per window. (if we had late data this key would not be unique) + * + *

      To access the window in a {@link DoFn}, add a {@link BoundedWindow} parameter. This will + * be automatically detected and populated with the window for the current element. */ - wordCounts.apply(ParDo.of(new FormatAsTableRowFn())) - .apply(BigQueryIO.Write - .to(getTableReference(options)) - .withSchema(getSchema()) - .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) - .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND)); + PCollection>> keyedByWindow = + wordCounts.apply( + ParDo.of( + new DoFn, KV>>() { + @ProcessElement + public void processElement(ProcessContext context, IntervalWindow window) { + context.output(KV.of(window, context.element())); + } + })); - PipelineResult result = pipeline.run(); + /** + * Concept #6: Format the results and write to a sharded file partitioned by window, using a + * simple ParDo operation. Because there may be failures followed by retries, the + * writes must be idempotent, but the details of writing to files is elided here. + */ + keyedByWindow + .apply(GroupByKey.>create()) + .apply(ParDo.of(new WriteWindowedFilesDoFn(output))); - // ExampleUtils will try to cancel the pipeline before the program exists. - exampleUtils.waitToFinish(result); + PipelineResult result = pipeline.run(); + try { + result.waitUntilFinish(); + } catch (Exception exc) { + result.cancel(); + } } + } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/WriteWindowedFilesDoFn.java b/examples/java/src/main/java/org/apache/beam/examples/common/WriteWindowedFilesDoFn.java new file mode 100644 index 000000000000..cd6baad44292 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/common/WriteWindowedFilesDoFn.java @@ -0,0 +1,77 @@ +/* + * 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.examples.common; + +import com.google.common.annotations.VisibleForTesting; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.values.KV; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; + +/** + * A {@link DoFn} that writes elements to files with names deterministically derived from the lower + * and upper bounds of their key (an {@link IntervalWindow}). + * + *

      This is test utility code, not for end-users, so examples can be focused + * on their primary lessons. + */ +public class WriteWindowedFilesDoFn + extends DoFn>>, Void> { + + static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); + static final Coder STRING_CODER = StringUtf8Coder.of(); + + private static DateTimeFormatter formatter = ISODateTimeFormat.hourMinute(); + + private final String output; + + public WriteWindowedFilesDoFn(String output) { + this.output = output; + } + + @VisibleForTesting + public static String fileForWindow(String output, IntervalWindow window) { + return String.format( + "%s-%s-%s", output, formatter.print(window.start()), formatter.print(window.end())); + } + + @ProcessElement + public void processElement(ProcessContext context) throws Exception { + // Build a file name from the window + IntervalWindow window = context.element().getKey(); + String outputShard = fileForWindow(output, window); + + // Open the file and write all the values + IOChannelFactory factory = IOChannelUtils.getFactory(outputShard); + OutputStream out = Channels.newOutputStream(factory.create(outputShard, "text/plain")); + for (KV wordCount : context.element().getValue()) { + STRING_CODER.encode( + wordCount.getKey() + ": " + wordCount.getValue(), out, Coder.Context.OUTER); + out.write(NEWLINE); + } + out.close(); + } +} diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index 5d77dd570395..e4570acb9533 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -17,37 +17,59 @@ */ package org.apache.beam.examples; -import java.io.IOException; +import static org.hamcrest.Matchers.equalTo; + +import com.google.api.client.util.Sleeper; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.Collections; import java.util.Date; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; +import org.apache.beam.examples.common.WriteWindowedFilesDoFn; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.testing.BigqueryMatcher; +import org.apache.beam.sdk.testing.FileChecksumMatcher; +import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.testing.StreamingIT; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.util.ExplicitShardedFile; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.ShardedFile; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -/** - * End-to-end integration test of {@link WindowedWordCount}. - */ +/** End-to-end integration test of {@link WindowedWordCount}. */ @RunWith(JUnit4.class) public class WindowedWordCountIT { private static final String DEFAULT_INPUT = "gs://apache-beam-samples/shakespeare/winterstale-personae"; - private static final String DEFAULT_OUTPUT_CHECKSUM = "cd5b52939257e12428a9fa085c32a84dd209b180"; + static final int MAX_READ_RETRIES = 4; + static final Duration DEFAULT_SLEEP_DURATION = Duration.standardSeconds(10L); + static final FluentBackoff BACK_OFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(DEFAULT_SLEEP_DURATION) + .withMaxRetries(MAX_READ_RETRIES); - /** - * Options for the {@link WindowedWordCount} Integration Test. - */ + /** Options for the {@link WindowedWordCount} Integration Test. */ public interface WindowedWordCountITOptions - extends WindowedWordCount.Options, TestPipelineOptions, StreamingOptions { - } + extends WindowedWordCount.Options, TestPipelineOptions, StreamingOptions {} @BeforeClass public static void setUp() { @@ -55,36 +77,140 @@ public static void setUp() { } @Test - public void testWindowedWordCountInBatch() throws IOException { - testWindowedWordCountPipeline(false /* isStreaming */); + public void testWindowedWordCountInBatch() throws Exception { + testWindowedWordCountPipeline(defaultOptions()); } @Test @Category(StreamingIT.class) - public void testWindowedWordCountInStreaming() throws IOException { - testWindowedWordCountPipeline(true /* isStreaming */); + public void testWindowedWordCountInStreaming() throws Exception { + testWindowedWordCountPipeline(streamingOptions()); } - private void testWindowedWordCountPipeline(boolean isStreaming) throws IOException { + private WindowedWordCountITOptions defaultOptions() throws Exception { WindowedWordCountITOptions options = TestPipeline.testingPipelineOptions().as(WindowedWordCountITOptions.class); - options.setStreaming(isStreaming); options.setInputFile(DEFAULT_INPUT); + options.setTestTimeoutSeconds(1200L); + + options.setMinTimestampMillis(0L); + options.setMinTimestampMillis(Duration.standardHours(1).getMillis()); + options.setWindowSize(10); + + options.setOutput( + IOChannelUtils.resolve( + options.getTempRoot(), + String.format("WindowedWordCountIT-%tF-% expectedOutputFiles = Lists.newArrayListWithCapacity(6); + for (int startMinute : ImmutableList.of(0, 10, 20, 30, 40, 50)) { + Instant windowStart = + new Instant(options.getMinTimestampMillis()).plus(Duration.standardMinutes(startMinute)); + expectedOutputFiles.add( + WriteWindowedFilesDoFn.fileForWindow( + outputPrefix, + new IntervalWindow(windowStart, windowStart.plus(Duration.standardMinutes(10))))); + } - // Note: currently unused because the example writes to BigQuery, but WindowedWordCount.Options - // are tightly coupled to WordCount.Options, where the option is required. - options.setOutput(IOChannelUtils.resolve( - options.getTempRoot(), - String.format("WindowedWordCountIT-%tF-% expectedWordCounts = new TreeMap<>(); + for (String line : + inputFile.readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff())) { + String[] words = line.split("[^a-zA-Z']+"); + + for (String word : words) { + if (!word.isEmpty()) { + expectedWordCounts.put(word, + MoreObjects.firstNonNull(expectedWordCounts.get(word), 0L) + 1L); + } + } + } - String query = String.format("SELECT word, SUM(count) FROM [%s:%s.%s] GROUP BY word", - options.getProject(), options.getBigQueryDataset(), options.getBigQueryTable()); options.setOnSuccessMatcher( - new BigqueryMatcher( - options.getAppName(), options.getProject(), query, DEFAULT_OUTPUT_CHECKSUM)); + new WordCountsMatcher(expectedWordCounts, new ExplicitShardedFile(expectedOutputFiles))); WindowedWordCount.main(TestPipeline.convertToArgs(options)); } + + /** + * A matcher that bakes in expected word counts, so they can be read directly via some other + * mechanism, and compares a sharded output file with the result. + */ + private static class WordCountsMatcher extends TypeSafeMatcher + implements SerializableMatcher { + + private static final Logger LOG = LoggerFactory.getLogger(FileChecksumMatcher.class); + + private final SortedMap expectedWordCounts; + private final ShardedFile outputFile; + private SortedMap actualCounts; + + public WordCountsMatcher(SortedMap expectedWordCounts, ShardedFile outputFile) { + this.expectedWordCounts = expectedWordCounts; + this.outputFile = outputFile; + } + + @Override + public boolean matchesSafely(PipelineResult pipelineResult) { + try { + // Load output data + List lines = + outputFile.readFilesWithRetries(Sleeper.DEFAULT, BACK_OFF_FACTORY.backoff()); + + // Since the windowing is nondeterministic we only check the sums + actualCounts = new TreeMap<>(); + for (String line : lines) { + String[] splits = line.split(": "); + String word = splits[0]; + long count = Long.parseLong(splits[1]); + + Long current = actualCounts.get(word); + if (current == null) { + actualCounts.put(word, count); + } else { + actualCounts.put(word, current + count); + } + } + + return actualCounts.equals(expectedWordCounts); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to read from sharded output: %s", outputFile)); + } + } + + @Override + public void describeTo(Description description) { + equalTo(expectedWordCounts).describeTo(description); + } + + @Override + public void describeMismatchSafely(PipelineResult pResult, Description description) { + equalTo(expectedWordCounts).describeMismatch(actualCounts, description); + } + } } From 3c2e550aa7a56fe16f278cb353314a1ee51dbfe3 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 12 Dec 2016 14:59:36 -0800 Subject: [PATCH 159/279] Add no-arg constructor for UnboundedReadFromBoundedSource This allows Kryo to work with the type, currently required by the Apex runner. --- .../beam/runners/core/UnboundedReadFromBoundedSource.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java index f3f93e1bccec..c8047251ce67 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java @@ -118,7 +118,12 @@ public void populateDisplayData(DisplayData.Builder builder) { public static class BoundedToUnboundedSourceAdapter extends UnboundedSource> { - private BoundedSource boundedSource; + @SuppressWarnings("unused") // for Kryo + private BoundedToUnboundedSourceAdapter() { + this.boundedSource = null; + } + + private final BoundedSource boundedSource; public BoundedToUnboundedSourceAdapter(BoundedSource boundedSource) { this.boundedSource = boundedSource; From 47cc2dca05daa4075093c414e13bf0cacaa77744 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 12 Dec 2016 16:33:53 -0800 Subject: [PATCH 160/279] Add Tests for Kryo Serialization of URFBS --- runners/core-java/pom.xml | 7 ++ .../UnboundedReadFromBoundedSourceTest.java | 97 +++++++++++++++++-- 2 files changed, 94 insertions(+), 10 deletions(-) diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml index 0e2b4b0ecbb5..bab9d5721384 100644 --- a/runners/core-java/pom.xml +++ b/runners/core-java/pom.xml @@ -152,6 +152,13 @@ + + com.esotericsoftware.kryo + kryo + 2.21 + test + + org.apache.beam diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java index 7fd8807585d1..8a1b70bcbed9 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java @@ -17,19 +17,28 @@ */ package org.apache.beam.runners.core; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; +import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import java.util.Random; @@ -44,11 +53,13 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.FileBasedSource; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Distinct; @@ -65,6 +76,7 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.objenesis.strategy.StdInstantiatorStrategy; /** * Unit tests for {@link UnboundedReadFromBoundedSource}. @@ -101,27 +113,92 @@ public void testBoundedToUnboundedSourceAdapter() throws Exception { PCollection output = p.apply(Read.from(unboundedSource).withMaxNumRecords(numElements)); - // Count == numElements PAssert - .thatSingleton(output.apply("Count", Count.globally())) - .isEqualTo(numElements); + .thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(numElements); // Unique count == numElements PAssert - .thatSingleton(output.apply(Distinct.create()) - .apply("UniqueCount", Count.globally())) - .isEqualTo(numElements); + .thatSingleton(output.apply(Distinct.create()) + .apply("UniqueCount", Count.globally())) + .isEqualTo(numElements); // Min == 0 PAssert - .thatSingleton(output.apply("Min", Min.globally())) - .isEqualTo(0L); + .thatSingleton(output.apply("Min", Min.globally())) + .isEqualTo(0L); // Max == numElements-1 PAssert - .thatSingleton(output.apply("Max", Max.globally())) - .isEqualTo(numElements - 1); + .thatSingleton(output.apply("Max", Max.globally())) + .isEqualTo(numElements - 1); + p.run(); } + @Test + public void testAdapterKryoSerializationNoMemoization() throws IOException { + long numElements = 100; + BoundedSource boundedSource = CountingSource.upTo(numElements); + UnboundedSource> unboundedSource = + new BoundedToUnboundedSourceAdapter<>(boundedSource); + + //Kryo instantiation + Kryo kryo = new Kryo(); + kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); + + //Serialization of object without any memoization + ByteArrayOutputStream adapterWithoutMemoizationBos = new ByteArrayOutputStream(); + try (Output output = new Output(adapterWithoutMemoizationBos)) { + kryo.writeObject(output, unboundedSource); + } + + // Copy empty and memoized variants of the Adapater + ByteArrayInputStream bisWithoutMemoization = + new ByteArrayInputStream(adapterWithoutMemoizationBos.toByteArray()); + BoundedToUnboundedSourceAdapter copiedWithoutMemoization = + kryo.readObject(new Input(bisWithoutMemoization), BoundedToUnboundedSourceAdapter.class); + + Source.Reader reader = + copiedWithoutMemoization.createReader(TestPipeline.testingPipelineOptions(), null); + List readLongs = SourceTestUtils.readFromUnstartedReader(reader); + assertThat(readLongs, hasSize((int) numElements)); + List expectedLongs = new ArrayList<>(); + for (int i = 0; i < numElements; i++) { + expectedLongs.add((long) i); + } + assertThat(readLongs, containsInAnyOrder(expectedLongs.toArray())); + } + + @Test + public void testAdapterKryoSerializationWithMemoization() throws IOException { + long numElements = 100; + BoundedSource boundedSource = CountingSource.upTo(numElements); + UnboundedSource> unboundedSource = + new BoundedToUnboundedSourceAdapter<>(boundedSource); + + //Kryo instantiation + Kryo kryo = new Kryo(); + kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); + // Serialization of object with memoized fields + ByteArrayOutputStream sourceWithMemoizationsOutStream = new ByteArrayOutputStream(); + try (Output output = new Output(sourceWithMemoizationsOutStream)) { + kryo.writeObject(output, unboundedSource); + } + + ByteArrayInputStream bisWithMemoization = + new ByteArrayInputStream(sourceWithMemoizationsOutStream.toByteArray()); + BoundedToUnboundedSourceAdapter copiedWithMemoization = + kryo.readObject(new Input(bisWithMemoization), BoundedToUnboundedSourceAdapter.class); + Source.Reader reader = + copiedWithMemoization.createReader(TestPipeline.testingPipelineOptions(), null); + List readLongs = SourceTestUtils.readFromUnstartedReader(reader); + assertThat(readLongs, hasSize((int) numElements)); + List expectedLongs = new ArrayList<>(); + for (int i = 0; i < numElements; i++) { + expectedLongs.add((long) i); + } + assertThat(readLongs, containsInAnyOrder(expectedLongs.toArray())); + } + @Test public void testCountingSourceToUnboundedCheckpoint() throws Exception { long numElements = 100; From 2b84b7bc08e4e351074f5cbd9f20e908295bd7d8 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 12 Dec 2016 17:41:59 -0800 Subject: [PATCH 161/279] [BEAM-1142] Upgrade maven-invoker to address maven bug ARCHETYPE-488. --- sdks/java/maven-archetypes/examples-java8/pom.xml | 7 +++++++ sdks/java/maven-archetypes/examples/pom.xml | 7 +++++++ sdks/java/maven-archetypes/starter/pom.xml | 7 +++++++ 3 files changed, 21 insertions(+) diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml index 5cf094e36aee..4aeb627ab78e 100644 --- a/sdks/java/maven-archetypes/examples-java8/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/pom.xml @@ -50,6 +50,13 @@ maven-archetype-plugin 2.4 + + + org.apache.maven.shared + maven-invoker + 2.2 + + + + + + + + + + + + + + + diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml index aedba9e689f3..93916e7a4ffe 100644 --- a/sdks/java/io/jms/pom.xml +++ b/sdks/java/io/jms/pom.xml @@ -37,19 +37,6 @@ - - - - - org.codehaus.mojo - findbugs-maven-plugin - - true - - - - - org.apache.maven.plugins diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java index 65a9189d2cea..15a1109df3c6 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java @@ -20,6 +20,8 @@ import java.io.Serializable; import java.util.Map; import java.util.Objects; + +import javax.annotation.Nullable; import javax.jms.Destination; /** @@ -31,7 +33,7 @@ public class JmsRecord implements Serializable { private final String jmsMessageID; private final long jmsTimestamp; private final String jmsCorrelationID; - private final Destination jmsReplyTo; + @Nullable private final Destination jmsReplyTo; private final Destination jmsDestination; private final int jmsDeliveryMode; private final boolean jmsRedelivered; @@ -45,7 +47,7 @@ public JmsRecord( String jmsMessageID, long jmsTimestamp, String jmsCorrelationID, - Destination jmsReplyTo, + @Nullable Destination jmsReplyTo, Destination jmsDestination, int jmsDeliveryMode, boolean jmsRedelivered, @@ -80,7 +82,7 @@ public String getJmsCorrelationID() { return jmsCorrelationID; } - public Destination getJmsReplyTo() { + public @Nullable Destination getJmsReplyTo() { return jmsReplyTo; } From db644318e1117d2a9afc3ed56270ad13e1186cfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 13 Dec 2016 15:09:03 +0100 Subject: [PATCH 167/279] Fix JDom malformed comment in Apex runner. --- runners/apex/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 9f1455a0adb0..74fe0b60d67d 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -75,7 +75,7 @@ runtime - + org.apache.beam beam-sdks-java-core From 6c8d93b018a040591b58e8f38d2a2442a7589692 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 13 Dec 2016 09:50:33 -0800 Subject: [PATCH 168/279] [BEAM-909] improve starter archetype --- .../main/resources/archetype-resources/pom.xml | 16 +++++++++++++--- .../resources/projects/basic/reference/pom.xml | 16 +++++++++++++--- 2 files changed, 26 insertions(+), 6 deletions(-) diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml index 4fae02cf7a02..efafeca14dc1 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml @@ -24,6 +24,10 @@ ${artifactId} ${version} + + 0.4.0-incubating-SNAPSHOT + + apache.snapshots @@ -69,14 +73,20 @@ org.apache.beam beam-sdks-java-core - 0.4.0-incubating-SNAPSHOT + ${beam.version} - + org.apache.beam beam-runners-direct-java - 0.4.0-incubating-SNAPSHOT + ${beam.version} runtime diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml index 4656e63c1d15..a86bd11af207 100644 --- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml @@ -24,6 +24,10 @@ basic 0.1 + + 0.4.0-incubating-SNAPSHOT + + apache.snapshots @@ -69,14 +73,20 @@ org.apache.beam beam-sdks-java-core - 0.4.0-incubating-SNAPSHOT + ${beam.version} - + org.apache.beam beam-runners-direct-java - 0.4.0-incubating-SNAPSHOT + ${beam.version} runtime From ddef4384bb527543c0ffce7c1e6cfcbac3a41b93 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 13 Dec 2016 09:48:25 -0800 Subject: [PATCH 169/279] Revert 91cc606 "This closes #1586": Kryo + UBRFBS The above commit was a merge of the following commits to master: 47cc2dc Add Tests for Kryo Serialization of URFBS 3c2e550 Add no-arg constructor for UnboundedReadFromBoundedSource Kryo support is, in fact, not needed for UnboundedReadFromBoundedSource, but we have a dependency issue where the Apex runner's dependency on Kryo was being overridden with an older version that does not support the @Bind annotations. All runners and poms are independently correct except the examples pom which merges the dependencies from all. --- runners/core-java/pom.xml | 7 -- .../core/UnboundedReadFromBoundedSource.java | 7 +- .../UnboundedReadFromBoundedSourceTest.java | 97 ++----------------- 3 files changed, 11 insertions(+), 100 deletions(-) diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml index bab9d5721384..0e2b4b0ecbb5 100644 --- a/runners/core-java/pom.xml +++ b/runners/core-java/pom.xml @@ -152,13 +152,6 @@ - - com.esotericsoftware.kryo - kryo - 2.21 - test - - org.apache.beam diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java index c8047251ce67..f3f93e1bccec 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java @@ -118,12 +118,7 @@ public void populateDisplayData(DisplayData.Builder builder) { public static class BoundedToUnboundedSourceAdapter extends UnboundedSource> { - @SuppressWarnings("unused") // for Kryo - private BoundedToUnboundedSourceAdapter() { - this.boundedSource = null; - } - - private final BoundedSource boundedSource; + private BoundedSource boundedSource; public BoundedToUnboundedSourceAdapter(BoundedSource boundedSource) { this.boundedSource = boundedSource; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java index 8a1b70bcbed9..7fd8807585d1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java @@ -17,28 +17,19 @@ */ package org.apache.beam.runners.core; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import java.util.Random; @@ -53,13 +44,11 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.FileBasedSource; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Distinct; @@ -76,7 +65,6 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.objenesis.strategy.StdInstantiatorStrategy; /** * Unit tests for {@link UnboundedReadFromBoundedSource}. @@ -113,92 +101,27 @@ public void testBoundedToUnboundedSourceAdapter() throws Exception { PCollection output = p.apply(Read.from(unboundedSource).withMaxNumRecords(numElements)); + // Count == numElements PAssert - .thatSingleton(output.apply("Count", Count.globally())) - .isEqualTo(numElements); + .thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(numElements); // Unique count == numElements PAssert - .thatSingleton(output.apply(Distinct.create()) - .apply("UniqueCount", Count.globally())) - .isEqualTo(numElements); + .thatSingleton(output.apply(Distinct.create()) + .apply("UniqueCount", Count.globally())) + .isEqualTo(numElements); // Min == 0 PAssert - .thatSingleton(output.apply("Min", Min.globally())) - .isEqualTo(0L); + .thatSingleton(output.apply("Min", Min.globally())) + .isEqualTo(0L); // Max == numElements-1 PAssert - .thatSingleton(output.apply("Max", Max.globally())) - .isEqualTo(numElements - 1); - + .thatSingleton(output.apply("Max", Max.globally())) + .isEqualTo(numElements - 1); p.run(); } - @Test - public void testAdapterKryoSerializationNoMemoization() throws IOException { - long numElements = 100; - BoundedSource boundedSource = CountingSource.upTo(numElements); - UnboundedSource> unboundedSource = - new BoundedToUnboundedSourceAdapter<>(boundedSource); - - //Kryo instantiation - Kryo kryo = new Kryo(); - kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); - - //Serialization of object without any memoization - ByteArrayOutputStream adapterWithoutMemoizationBos = new ByteArrayOutputStream(); - try (Output output = new Output(adapterWithoutMemoizationBos)) { - kryo.writeObject(output, unboundedSource); - } - - // Copy empty and memoized variants of the Adapater - ByteArrayInputStream bisWithoutMemoization = - new ByteArrayInputStream(adapterWithoutMemoizationBos.toByteArray()); - BoundedToUnboundedSourceAdapter copiedWithoutMemoization = - kryo.readObject(new Input(bisWithoutMemoization), BoundedToUnboundedSourceAdapter.class); - - Source.Reader reader = - copiedWithoutMemoization.createReader(TestPipeline.testingPipelineOptions(), null); - List readLongs = SourceTestUtils.readFromUnstartedReader(reader); - assertThat(readLongs, hasSize((int) numElements)); - List expectedLongs = new ArrayList<>(); - for (int i = 0; i < numElements; i++) { - expectedLongs.add((long) i); - } - assertThat(readLongs, containsInAnyOrder(expectedLongs.toArray())); - } - - @Test - public void testAdapterKryoSerializationWithMemoization() throws IOException { - long numElements = 100; - BoundedSource boundedSource = CountingSource.upTo(numElements); - UnboundedSource> unboundedSource = - new BoundedToUnboundedSourceAdapter<>(boundedSource); - - //Kryo instantiation - Kryo kryo = new Kryo(); - kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); - // Serialization of object with memoized fields - ByteArrayOutputStream sourceWithMemoizationsOutStream = new ByteArrayOutputStream(); - try (Output output = new Output(sourceWithMemoizationsOutStream)) { - kryo.writeObject(output, unboundedSource); - } - - ByteArrayInputStream bisWithMemoization = - new ByteArrayInputStream(sourceWithMemoizationsOutStream.toByteArray()); - BoundedToUnboundedSourceAdapter copiedWithMemoization = - kryo.readObject(new Input(bisWithMemoization), BoundedToUnboundedSourceAdapter.class); - Source.Reader reader = - copiedWithMemoization.createReader(TestPipeline.testingPipelineOptions(), null); - List readLongs = SourceTestUtils.readFromUnstartedReader(reader); - assertThat(readLongs, hasSize((int) numElements)); - List expectedLongs = new ArrayList<>(); - for (int i = 0; i < numElements; i++) { - expectedLongs.add((long) i); - } - assertThat(readLongs, containsInAnyOrder(expectedLongs.toArray())); - } - @Test public void testCountingSourceToUnboundedCheckpoint() throws Exception { long numElements = 100; From f32f156ad9443ff63664dc62e49f6e48ad81baeb Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 13 Dec 2016 10:34:47 -0800 Subject: [PATCH 170/279] Fix version of Kryo in examples/java jenkins-precommit profile --- examples/java/pom.xml | 18 ++++++++++++++++++ pom.xml | 1 + runners/apex/pom.xml | 2 +- 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 01d1695a81f5..e541edb0663f 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -131,6 +131,24 @@ false true + + + + + com.esotericsoftware.kryo + kryo + ${apex.kryo.version} + runtime + + + diff --git a/pom.xml b/pom.xml index 35a055cea55c..32bcab069ae0 100644 --- a/pom.xml +++ b/pom.xml @@ -101,6 +101,7 @@ + 2.24.0 1.8.1 v2-rev295-1.22.0 v1-rev6-1.22.0 diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 9f1455a0adb0..6c23f5228850 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -221,7 +221,7 @@ org.apache.apex:apex-api:jar:3.5.0 org.apache.commons:commons-lang3::3.1 commons-io:commons-io:jar:2.4 - com.esotericsoftware.kryo:kryo::2.24.0 + com.esotericsoftware.kryo:kryo::${apex.kryo.version} com.datatorrent:netlet::1.3.0 org.slf4j:slf4j-api:jar:1.7.14 org.apache.hadoop:hadoop-common:jar:2.6.0 From c8dd9989c49aab1cede743a45cad321b9b9b5a2d Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 13 Dec 2016 13:34:22 -0800 Subject: [PATCH 171/279] [maven-release-plugin] prepare branch release-0.4.0-incubating --- pom.xml | 2 +- runners/apex/pom.xml | 6 ++---- sdks/java/maven-archetypes/examples-java8/pom.xml | 4 +--- sdks/java/maven-archetypes/examples/pom.xml | 4 +--- 4 files changed, 5 insertions(+), 11 deletions(-) diff --git a/pom.xml b/pom.xml index 32bcab069ae0..e860c7a6d9ae 100644 --- a/pom.xml +++ b/pom.xml @@ -48,7 +48,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git https://git-wip-us.apache.org/repos/asf?p=incubator-beam.git;a=summary - release-0.2.0-incubating + release-0.4.0-incubating diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index a47cf74ca1eb..42d7212f9646 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -267,7 +265,7 @@ - + diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml index 4aeb627ab78e..53ad69899716 100644 --- a/sdks/java/maven-archetypes/examples-java8/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml index 72d85f9413f4..3fcd7d42cf7f 100644 --- a/sdks/java/maven-archetypes/examples/pom.xml +++ b/sdks/java/maven-archetypes/examples/pom.xml @@ -15,9 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 From 596f255bcbfd1d14edbf17aea15d5313b7f34b80 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 13 Dec 2016 13:35:15 -0800 Subject: [PATCH 172/279] [maven-release-plugin] prepare for next development iteration --- examples/java/pom.xml | 2 +- examples/java8/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 4 ++-- runners/apex/pom.xml | 2 +- runners/core-java/pom.xml | 2 +- runners/direct-java/pom.xml | 2 +- runners/flink/examples/pom.xml | 2 +- runners/flink/pom.xml | 2 +- runners/flink/runner/pom.xml | 2 +- runners/google-cloud-dataflow-java/pom.xml | 2 +- runners/pom.xml | 2 +- runners/spark/pom.xml | 2 +- sdks/java/build-tools/pom.xml | 2 +- sdks/java/core/pom.xml | 2 +- sdks/java/extensions/join-library/pom.xml | 2 +- sdks/java/extensions/pom.xml | 2 +- sdks/java/extensions/sorter/pom.xml | 2 +- sdks/java/io/google-cloud-platform/pom.xml | 2 +- sdks/java/io/hdfs/pom.xml | 2 +- sdks/java/io/jdbc/pom.xml | 2 +- sdks/java/io/jms/pom.xml | 2 +- sdks/java/io/kafka/pom.xml | 2 +- sdks/java/io/kinesis/pom.xml | 2 +- sdks/java/io/mongodb/pom.xml | 2 +- sdks/java/io/pom.xml | 2 +- sdks/java/java8tests/pom.xml | 2 +- sdks/java/maven-archetypes/examples-java8/pom.xml | 2 +- sdks/java/maven-archetypes/examples/pom.xml | 2 +- sdks/java/maven-archetypes/pom.xml | 2 +- sdks/java/maven-archetypes/starter/pom.xml | 2 +- sdks/java/pom.xml | 2 +- sdks/pom.xml | 2 +- 33 files changed, 34 insertions(+), 34 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index e541edb0663f..dc86532d2265 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-examples-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 8591955e4caf..370d79f2bdf3 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-examples-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb42861bad12..adfbaa97e347 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index e860c7a6d9ae..b57be984d4d3 100644 --- a/pom.xml +++ b/pom.xml @@ -34,7 +34,7 @@ http://beam.incubator.apache.org 2016 - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT @@ -48,7 +48,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-beam.git https://git-wip-us.apache.org/repos/asf?p=incubator-beam.git;a=summary - release-0.4.0-incubating + release-0.2.0-incubating diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 42d7212f9646..3dbfb136a5eb 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml index 0e2b4b0ecbb5..de7dcf5741f4 100644 --- a/runners/core-java/pom.xml +++ b/runners/core-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index c47f53270576..3578907e550c 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index dccb5cbb3718..b623d58168d5 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-flink-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index f93af85a40cf..7eab021ee5cd 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 37de89962df9..daf662326009 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-flink-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 1543c0e00937..ea99df115481 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/pom.xml b/runners/pom.xml index 6f309266a9a9..fb84164b266b 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 86e9039a80c8..6795b8319639 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/build-tools/pom.xml b/sdks/java/build-tools/pom.xml index b911328968d2..4c0a749c3980 100644 --- a/sdks/java/build-tools/pom.xml +++ b/sdks/java/build-tools/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../../../pom.xml diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index ad848469c413..b02cd1dc3829 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml index 554792a1d01d..def0340c81c5 100644 --- a/sdks/java/extensions/join-library/pom.xml +++ b/sdks/java/extensions/join-library/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml index fc90edf7e32d..01645d26e3b1 100644 --- a/sdks/java/extensions/pom.xml +++ b/sdks/java/extensions/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/sorter/pom.xml b/sdks/java/extensions/sorter/pom.xml index 8d1e89de32e5..c5557c490daa 100644 --- a/sdks/java/extensions/sorter/pom.xml +++ b/sdks/java/extensions/sorter/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 4be00a9c86c5..319c1672c280 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index bd3445195534..1b62f1a99c8c 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index 428695c0f7b8..44c7abd39f96 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml index 93916e7a4ffe..db7a2d9f7e8c 100644 --- a/sdks/java/io/jms/pom.xml +++ b/sdks/java/io/jms/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index cba1eb3e6657..2637449be0ee 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml index 5bf41f0cbba2..29e9e5fedf81 100644 --- a/sdks/java/io/kinesis/pom.xml +++ b/sdks/java/io/kinesis/pom.xml @@ -21,7 +21,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml index 4c0066eed5b3..56de6b3a81b2 100644 --- a/sdks/java/io/mongodb/pom.xml +++ b/sdks/java/io/mongodb/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 3750202418b9..47afe92c00e2 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml index d9dfdce66d40..8545de4b5c75 100644 --- a/sdks/java/java8tests/pom.xml +++ b/sdks/java/java8tests/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml index 53ad69899716..5f2720759b3c 100644 --- a/sdks/java/maven-archetypes/examples-java8/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml index 3fcd7d42cf7f..e819c8c6cfb3 100644 --- a/sdks/java/maven-archetypes/examples/pom.xml +++ b/sdks/java/maven-archetypes/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml index be9ad6a752ac..28e9fe971a7a 100644 --- a/sdks/java/maven-archetypes/pom.xml +++ b/sdks/java/maven-archetypes/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/starter/pom.xml b/sdks/java/maven-archetypes/starter/pom.xml index 8ee943808b6a..5b2140756bfe 100644 --- a/sdks/java/maven-archetypes/starter/pom.xml +++ b/sdks/java/maven-archetypes/starter/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 0907c1a18dfe..1ab3452103bf 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml diff --git a/sdks/pom.xml b/sdks/pom.xml index 29ccd377fb8b..bfdfcd9d6be6 100644 --- a/sdks/pom.xml +++ b/sdks/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT ../pom.xml From d323d5d9f0396450ac78cfc747a7bc80695cbdf7 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 12 Dec 2016 13:48:24 -0800 Subject: [PATCH 173/279] Update Dataflow worker to beam-master-20161212 --- .../org/apache/beam/runners/dataflow/dataflow.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index 1eae8cb9f055..d43881dfe304 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -18,6 +18,6 @@ environment.major.version=6 -worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161205 +worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161212 -worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161205 +worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161212 From 267edaaab1d5d50e71cce8f7f417b31ac029ba74 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 14:22:21 -0800 Subject: [PATCH 174/279] Move PerKeyCombineFnRunner to runners-core --- .../org/apache/beam/runners/core}/PerKeyCombineFnRunner.java | 3 ++- .../org/apache/beam/runners/core/PerKeyCombineFnRunners.java | 1 - .../functions/FlinkMergingNonShuffleReduceFunction.java | 2 +- .../functions/FlinkMergingPartialReduceFunction.java | 2 +- .../translation/functions/FlinkMergingReduceFunction.java | 2 +- .../translation/functions/FlinkPartialReduceFunction.java | 2 +- .../flink/translation/functions/FlinkReduceFunction.java | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/PerKeyCombineFnRunner.java (98%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunner.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunner.java index c1a93b706b5b..a927ecd31312 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import java.io.Serializable; import java.util.Collection; @@ -23,6 +23,7 @@ import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.SideInputReader; /** * An interface that runs a {@link PerKeyCombineFn} with unified APIs. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java index 6f0ff96ef485..34d711bc241d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PerKeyCombineFnRunners.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CombineContextFactory; -import org.apache.beam.sdk.util.PerKeyCombineFnRunner; import org.apache.beam.sdk.util.SideInputReader; /** diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index 3db98a31e6ca..041d0e8684cf 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunner; import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -33,7 +34,6 @@ 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.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java index ea0669a94b2b..fef7921098bf 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunner; import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -31,7 +32,6 @@ 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.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java index 9a4aadc83e44..59163e961430 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunner; import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; @@ -33,7 +34,6 @@ 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.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; 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 ca804615e4f7..8b6ec3a0ea1c 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 @@ -24,6 +24,7 @@ import java.util.Comparator; import java.util.Iterator; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunner; import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -32,7 +33,6 @@ 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.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; 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 b4d003c9e8a5..fb5c90cb85b1 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 @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.PerKeyCombineFnRunner; import org.apache.beam.runners.core.PerKeyCombineFnRunners; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -34,7 +35,6 @@ 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.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; From 5d6a50c63360d8e706c095b73b55734390615135 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 13 Dec 2016 13:43:58 -0800 Subject: [PATCH 175/279] Update Maven Archetype versions after cutting the release branch --- .../src/main/resources/archetype-resources/pom.xml | 6 +++--- .../examples/src/main/resources/archetype-resources/pom.xml | 2 +- .../starter/src/main/resources/archetype-resources/pom.xml | 2 +- .../src/test/resources/projects/basic/reference/pom.xml | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index 693850ae34fb..a4404242a83b 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -27,7 +27,7 @@ jar - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT @@ -177,14 +177,14 @@ org.apache.beam beam-sdks-java-core - 0.4.0-incubating-SNAPSHOT + ${beam.version} org.apache.beam beam-sdks-java-io-google-cloud-platform - 0.4.0-incubating-SNAPSHOT + ${beam.version} diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 9bbc6463e489..b6eab15b4b62 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -27,7 +27,7 @@ jar - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml index efafeca14dc1..4a6e89ddaa6e 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml @@ -25,7 +25,7 @@ ${version} - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml index a86bd11af207..97f9e7210c12 100644 --- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml @@ -25,7 +25,7 @@ 0.1 - 0.4.0-incubating-SNAPSHOT + 0.5.0-SNAPSHOT From 25215889381f7da61766054af68c84ffed4c0c71 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 13 Dec 2016 14:27:55 -0800 Subject: [PATCH 176/279] Update version from 0.5.0-SNAPSHOT to 0.5.0-incubating-SNAPSHOT --- examples/java/pom.xml | 2 +- examples/java8/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- runners/apex/pom.xml | 2 +- runners/core-java/pom.xml | 2 +- runners/direct-java/pom.xml | 2 +- runners/flink/examples/pom.xml | 2 +- runners/flink/pom.xml | 2 +- runners/flink/runner/pom.xml | 2 +- runners/google-cloud-dataflow-java/pom.xml | 2 +- runners/pom.xml | 2 +- runners/spark/pom.xml | 2 +- sdks/java/build-tools/pom.xml | 2 +- sdks/java/core/pom.xml | 2 +- sdks/java/extensions/join-library/pom.xml | 2 +- sdks/java/extensions/pom.xml | 2 +- sdks/java/extensions/sorter/pom.xml | 2 +- sdks/java/io/google-cloud-platform/pom.xml | 2 +- sdks/java/io/hdfs/pom.xml | 2 +- sdks/java/io/jdbc/pom.xml | 2 +- sdks/java/io/jms/pom.xml | 2 +- sdks/java/io/kafka/pom.xml | 2 +- sdks/java/io/kinesis/pom.xml | 2 +- sdks/java/io/mongodb/pom.xml | 2 +- sdks/java/io/pom.xml | 2 +- sdks/java/java8tests/pom.xml | 2 +- sdks/java/maven-archetypes/examples-java8/pom.xml | 2 +- .../src/main/resources/archetype-resources/pom.xml | 2 +- sdks/java/maven-archetypes/examples/pom.xml | 2 +- .../examples/src/main/resources/archetype-resources/pom.xml | 2 +- sdks/java/maven-archetypes/pom.xml | 2 +- sdks/java/maven-archetypes/starter/pom.xml | 2 +- .../starter/src/main/resources/archetype-resources/pom.xml | 5 ++--- .../src/test/resources/projects/basic/reference/pom.xml | 2 +- sdks/java/pom.xml | 2 +- sdks/pom.xml | 2 +- 37 files changed, 38 insertions(+), 39 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index dc86532d2265..f1e0fe16ccc0 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-examples-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 370d79f2bdf3..1abf6fdd5974 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-examples-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index adfbaa97e347..3e0fbddfee1b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index b57be984d4d3..4faa971eeeac 100644 --- a/pom.xml +++ b/pom.xml @@ -34,7 +34,7 @@ http://beam.incubator.apache.org 2016 - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 3dbfb136a5eb..f71637c66fab 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml index de7dcf5741f4..b5c610b97a21 100644 --- a/runners/core-java/pom.xml +++ b/runners/core-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 3578907e550c..5da569259bdf 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index b623d58168d5..314584870a9e 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-flink-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 7eab021ee5cd..8d76ab0495f0 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index daf662326009..09773e106e6e 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-flink-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index ea99df115481..4c2378c0a122 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/pom.xml b/runners/pom.xml index fb84164b266b..426593990074 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 6795b8319639..e8fffa2b91e3 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-runners-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/build-tools/pom.xml b/sdks/java/build-tools/pom.xml index 4c0a749c3980..bf89b7d0768d 100644 --- a/sdks/java/build-tools/pom.xml +++ b/sdks/java/build-tools/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../../../pom.xml diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index b02cd1dc3829..d8d2c2670c9c 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml index def0340c81c5..f7ad06ae75a1 100644 --- a/sdks/java/extensions/join-library/pom.xml +++ b/sdks/java/extensions/join-library/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml index 01645d26e3b1..bca715a5b9c0 100644 --- a/sdks/java/extensions/pom.xml +++ b/sdks/java/extensions/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/extensions/sorter/pom.xml b/sdks/java/extensions/sorter/pom.xml index c5557c490daa..a99a793c07ba 100644 --- a/sdks/java/extensions/sorter/pom.xml +++ b/sdks/java/extensions/sorter/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-extensions-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 319c1672c280..d3b5fed573fb 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 1b62f1a99c8c..772276bf3113 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index 44c7abd39f96..6e3695eac272 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml index db7a2d9f7e8c..bca0152f73b4 100644 --- a/sdks/java/io/jms/pom.xml +++ b/sdks/java/io/jms/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index 2637449be0ee..f109e1351f6e 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml index 29e9e5fedf81..48e68308b7ee 100644 --- a/sdks/java/io/kinesis/pom.xml +++ b/sdks/java/io/kinesis/pom.xml @@ -21,7 +21,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml index 56de6b3a81b2..dde54a844a6c 100644 --- a/sdks/java/io/mongodb/pom.xml +++ b/sdks/java/io/mongodb/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 47afe92c00e2..12ede66660c3 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml index 8545de4b5c75..616e2ef0b1c3 100644 --- a/sdks/java/java8tests/pom.xml +++ b/sdks/java/java8tests/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml index 5f2720759b3c..5b0b30464f61 100644 --- a/sdks/java/maven-archetypes/examples-java8/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index a4404242a83b..65068c1f2267 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -27,7 +27,7 @@ jar - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml index e819c8c6cfb3..7a924ea03215 100644 --- a/sdks/java/maven-archetypes/examples/pom.xml +++ b/sdks/java/maven-archetypes/examples/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index b6eab15b4b62..9ec2223e46d1 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -27,7 +27,7 @@ jar - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml index 28e9fe971a7a..e91fd05649e0 100644 --- a/sdks/java/maven-archetypes/pom.xml +++ b/sdks/java/maven-archetypes/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/starter/pom.xml b/sdks/java/maven-archetypes/starter/pom.xml index 5b2140756bfe..dd1f313687cd 100644 --- a/sdks/java/maven-archetypes/starter/pom.xml +++ b/sdks/java/maven-archetypes/starter/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-maven-archetypes-parent - 0.5.0-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml index 4a6e89ddaa6e..c59ffee2ef30 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml @@ -1,5 +1,4 @@ - - From 00b961df2e6086ea9f5cb8b9b8fb747739d33670 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 14 Dec 2016 13:12:43 -0800 Subject: [PATCH 188/279] Fix NPE in StatefulParDoEvaluatorFactoryTest mocking --- .../runners/direct/StatefulParDoEvaluatorFactoryTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 7c086a1b629e..326310bf6db9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -31,6 +31,7 @@ import com.google.common.collect.Lists; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -94,6 +95,11 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable { public void setup() { MockitoAnnotations.initMocks(this); when((StateInternals) mockStepContext.stateInternals()).thenReturn(stateInternals); + when(mockEvaluationContext.createSideInputReader(anyList())) + .thenReturn( + SideInputContainer.create( + mockEvaluationContext, Collections.>emptyList()) + .createReaderForViews(Collections.>emptyList())); } @Test From 028408f5a9955879f03e5bb65c54813922ee4672 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 13 Dec 2016 18:29:17 -0800 Subject: [PATCH 189/279] [BEAM-1153] GcsUtil: use non-batch API for single file size requests. --- .../org/apache/beam/sdk/util/GcsUtil.java | 29 ++++++++- .../org/apache/beam/sdk/util/GcsUtilTest.java | 65 ++++++++++++++++++- 2 files changed, 92 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java index 2edb1d6932c8..dcdba464bca5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java @@ -267,7 +267,34 @@ Integer getUploadBufferSizeBytes() { * if the resource does not exist. */ public long fileSize(GcsPath path) throws IOException { - return fileSizes(ImmutableList.of(path)).get(0); + return fileSize( + path, + BACKOFF_FACTORY.backoff(), + Sleeper.DEFAULT); + } + + /** + * Returns the file size from GCS or throws {@link FileNotFoundException} + * if the resource does not exist. + */ + @VisibleForTesting + long fileSize(GcsPath path, BackOff backoff, Sleeper sleeper) throws IOException { + Storage.Objects.Get getObject = + storageClient.objects().get(path.getBucket(), path.getObject()); + try { + StorageObject object = ResilientOperation.retry( + ResilientOperation.getGoogleRequestCallable(getObject), + backoff, + RetryDeterminer.SOCKET_ERRORS, + IOException.class, + sleeper); + return object.getSize().longValue(); + } catch (Exception e) { + if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) { + throw new FileNotFoundException(path.toString()); + } + throw new IOException("Unable to get file size", e); + } } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java index c8ed402f3cfb..6ca87f9112c3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java @@ -57,6 +57,7 @@ import com.google.common.collect.Lists; import java.io.FileNotFoundException; import java.io.IOException; +import java.math.BigInteger; import java.net.SocketTimeoutException; import java.nio.channels.SeekableByteChannel; import java.nio.file.AccessDeniedException; @@ -320,7 +321,69 @@ public void testAccessDeniedObjectThrowsIOException() throws IOException { } @Test - public void testGetSizeBytesWhenFileNotFound() throws Exception { + public void testFileSizeNonBatch() throws Exception { + GcsOptions pipelineOptions = gcsOptionsWithTestCredential(); + GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); + + Storage mockStorage = Mockito.mock(Storage.class); + gcsUtil.setStorageClient(mockStorage); + + Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); + Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class); + + when(mockStorage.objects()).thenReturn(mockStorageObjects); + when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet); + when(mockStorageGet.execute()).thenReturn( + new StorageObject().setSize(BigInteger.valueOf(1000))); + + assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"))); + } + + @Test + public void testFileSizeWhenFileNotFoundNonBatch() throws Exception { + MockLowLevelHttpResponse notFoundResponse = new MockLowLevelHttpResponse(); + notFoundResponse.setContent(""); + notFoundResponse.setStatusCode(HttpStatusCodes.STATUS_CODE_NOT_FOUND); + + MockHttpTransport mockTransport = + new MockHttpTransport.Builder().setLowLevelHttpResponse(notFoundResponse).build(); + + GcsOptions pipelineOptions = gcsOptionsWithTestCredential(); + GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); + + gcsUtil.setStorageClient(new Storage(mockTransport, Transport.getJsonFactory(), null)); + + thrown.expect(FileNotFoundException.class); + gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject")); + } + + @Test + public void testRetryFileSizeNonBatch() throws IOException { + GcsOptions pipelineOptions = gcsOptionsWithTestCredential(); + GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); + + Storage mockStorage = Mockito.mock(Storage.class); + gcsUtil.setStorageClient(mockStorage); + + Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); + Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class); + + BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff(); + + when(mockStorage.objects()).thenReturn(mockStorageObjects); + when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet); + when(mockStorageGet.execute()) + .thenThrow(new SocketTimeoutException("SocketException")) + .thenThrow(new SocketTimeoutException("SocketException")) + .thenReturn(new StorageObject().setSize(BigInteger.valueOf(1000))); + + assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"), + mockBackOff, new FastNanoClockAndSleeper())); + assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis()); + } + + @Test + public void testGetSizeBytesWhenFileNotFoundBatch() throws Exception { JsonFactory jsonFactory = new JacksonFactory(); String contentBoundary = "batch_foobarbaz"; From de109d5b4c7693e935b68233c32e70f3f6b3d513 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 14 Dec 2016 14:29:30 -0800 Subject: [PATCH 190/279] [BEAM-1154] Get side input from proper window in ReduceFn --- .../runners/core/ReduceFnContextFactory.java | 16 +-- .../beam/runners/core/ReduceFnRunnerTest.java | 133 +++++++++--------- .../sdk/transforms/CombineWithContext.java | 2 +- 3 files changed, 78 insertions(+), 73 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index c5bda9b46e91..c71897dadc98 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.ActiveWindowSet; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimeDomain; @@ -98,11 +97,7 @@ private StateAccessorImpl stateAccessor(W window, StateStyle style) { activeWindows, windowingStrategy.getWindowFn().windowCoder(), stateInternals, - stateContextFromComponents( - options, - sideInputReader, - window, - windowingStrategy.getWindowFn()), + stateContextFromComponents(options, sideInputReader, window), style); } @@ -512,8 +507,7 @@ public Timers timers() { private static StateContext stateContextFromComponents( @Nullable final PipelineOptions options, final SideInputReader sideInputReader, - final W mainInputWindow, - final WindowFn windowFn) { + final W mainInputWindow) { if (options == null) { return StateContexts.nullContext(); } else { @@ -526,7 +520,11 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { - return sideInputReader.get(view, windowFn.getSideInputWindow(mainInputWindow)); + return sideInputReader.get( + view, + view.getWindowingStrategyInternal() + .getWindowFn() + .getSideInputWindow(mainInputWindow)); } @Override diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index ba5756798ee5..4abfc9ab1cc8 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -17,13 +17,14 @@ */ package org.apache.beam.runners.core; -import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.runners.core.WindowMatchers.isSingleWindowedValue; import static org.apache.beam.runners.core.WindowMatchers.isWindowedValue; +import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -36,7 +37,6 @@ import static org.mockito.Mockito.withSettings; import com.google.common.collect.Iterables; -import java.util.Iterator; import java.util.List; import org.apache.beam.runners.core.triggers.TriggerStateMachine; import org.apache.beam.sdk.coders.VarIntCoder; @@ -348,49 +348,67 @@ public void testOnElementCombiningAccumulating() throws Exception { @Test public void testOnElementCombiningWithContext() throws Exception { - Integer expectedValue = 5; - WindowingStrategy windowingStrategy = WindowingStrategy - .of(FixedWindows.of(Duration.millis(10))) - .withMode(AccumulationMode.DISCARDING_FIRED_PANES) - .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()) - .withAllowedLateness(Duration.millis(100)); + // Create values at timestamps 0 .. 8, windowed into fixed windows of 2. + // Side input windowed into fixed windows of 4: + // main: [ 0 1 ] [ 2 3 ] [ 4 5 ] [ 6 7 ] + // side: [ 100 ] [ 104 ] + // Combine using a CombineFn "side input + sum(main inputs)". + final int firstWindowSideInput = 100; + final int secondWindowSideInput = 104; + final Integer expectedValue = firstWindowSideInput; + WindowingStrategy mainInputWindowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.millis(2))) + .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES); + + WindowingStrategy sideInputWindowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.millis(4))); TestOptions options = PipelineOptionsFactory.as(TestOptions.class); - options.setValue(5); + options.setValue(expectedValue); when(mockSideInputReader.contains(Matchers.>any())).thenReturn(true); when(mockSideInputReader.get( - Matchers.>any(), any(BoundedWindow.class))).thenReturn(5); + Matchers.>any(), any(BoundedWindow.class))) + .then( + new Answer() { + @Override + public Integer answer(InvocationOnMock invocation) throws Throwable { + IntervalWindow sideInputWindow = (IntervalWindow) invocation.getArguments()[1]; + long startMs = sideInputWindow.start().getMillis(); + long endMs = sideInputWindow.end().getMillis(); + // Window should have been produced by sideInputWindowingStrategy. + assertThat(startMs, anyOf(equalTo(0L), equalTo(4L))); + assertThat(endMs - startMs, equalTo(4L)); + // If startMs == 4 (second window), equal to secondWindowSideInput. + return firstWindowSideInput + (int) startMs; + } + }); @SuppressWarnings({"rawtypes", "unchecked", "unused"}) Object suppressWarningsVar = when(mockView.getWindowingStrategyInternal()) - .thenReturn((WindowingStrategy) windowingStrategy); + .thenReturn((WindowingStrategy) sideInputWindowingStrategy); SumAndVerifyContextFn combineFn = new SumAndVerifyContextFn(mockView, expectedValue); - // Test basic execution of a trigger using a non-combining window set and discarding mode. ReduceFnTester tester = ReduceFnTester.combining( - windowingStrategy, mockTriggerStateMachine, combineFn.asKeyedFn(), + mainInputWindowingStrategy, mockTriggerStateMachine, combineFn.asKeyedFn(), VarIntCoder.of(), options, mockSideInputReader); - injectElement(tester, 2); - - when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - injectElement(tester, 3); - when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - triggerShouldFinish(mockTriggerStateMachine); - injectElement(tester, 4); - - // This element shouldn't be seen, because the trigger has finished - injectElement(tester, 6); + for (int i = 0; i < 8; ++i) { + injectElement(tester, i); + } assertThat( tester.extractOutput(), contains( - isSingleWindowedValue(equalTo(5), 2, 0, 10), - isSingleWindowedValue(equalTo(4), 4, 0, 10))); - assertTrue(tester.isMarkedFinished(firstWindow)); - tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); + isSingleWindowedValue(equalTo(0 + firstWindowSideInput), 1, 0, 2), + isSingleWindowedValue(equalTo(0 + 1 + firstWindowSideInput), 1, 0, 2), + isSingleWindowedValue(equalTo(2 + firstWindowSideInput), 3, 2, 4), + isSingleWindowedValue(equalTo(2 + 3 + firstWindowSideInput), 3, 2, 4), + isSingleWindowedValue(equalTo(4 + secondWindowSideInput), 5, 4, 6), + isSingleWindowedValue(equalTo(4 + 5 + secondWindowSideInput), 5, 4, 6), + isSingleWindowedValue(equalTo(6 + secondWindowSideInput), 7, 6, 8), + isSingleWindowedValue(equalTo(6 + 7 + secondWindowSideInput), 7, 6, 8))); } @Test @@ -1424,7 +1442,8 @@ public void setGarbageCollectionHoldOnLateElements() throws Exception { assertEquals(2, output.size()); } - private static class SumAndVerifyContextFn extends CombineFnWithContext { + private static class SumAndVerifyContextFn + extends CombineFnWithContext { private final PCollectionView view; private final int expectedValue; @@ -1433,50 +1452,38 @@ private SumAndVerifyContextFn(PCollectionView view, int expectedValue) this.view = view; this.expectedValue = expectedValue; } - @Override - public int[] createAccumulator(Context c) { - checkArgument( - c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue); - checkArgument(c.sideInput(view) == expectedValue); - return wrap(0); + + private void verifyContext(Context c) { + assertThat(expectedValue, equalTo(c.getPipelineOptions().as(TestOptions.class).getValue())); + assertThat(c.sideInput(view), greaterThanOrEqualTo(100)); } @Override - public int[] addInput(int[] accumulator, Integer input, Context c) { - checkArgument( - c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue); - checkArgument(c.sideInput(view) == expectedValue); - accumulator[0] += input.intValue(); - return accumulator; + public Integer createAccumulator(Context c) { + verifyContext(c); + return 0; } @Override - public int[] mergeAccumulators(Iterable accumulators, Context c) { - checkArgument( - c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue); - checkArgument(c.sideInput(view) == expectedValue); - Iterator iter = accumulators.iterator(); - if (!iter.hasNext()) { - return createAccumulator(c); - } else { - int[] running = iter.next(); - while (iter.hasNext()) { - running[0] += iter.next()[0]; - } - return running; - } + public Integer addInput(Integer accumulator, Integer input, Context c) { + verifyContext(c); + return accumulator + input; } @Override - public Integer extractOutput(int[] accumulator, Context c) { - checkArgument( - c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue); - checkArgument(c.sideInput(view) == expectedValue); - return accumulator[0]; + public Integer mergeAccumulators(Iterable accumulators, Context c) { + verifyContext(c); + int res = 0; + for (Integer accum : accumulators) { + res += accum; + } + return res; } - private int[] wrap(int value) { - return new int[] { value }; + @Override + public Integer extractOutput(Integer accumulator, Context c) { + verifyContext(c); + return accumulator + c.sideInput(view); } } @@ -1484,7 +1491,7 @@ private int[] wrap(int value) { * A {@link PipelineOptions} to test combining with context. */ public interface TestOptions extends PipelineOptions { - Integer getValue(); - void setValue(Integer value); + int getValue(); + void setValue(int value); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java index 7ac952c9587d..cd0600a7a951 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java @@ -48,7 +48,7 @@ public abstract static class Context { /** * Returns the value of the side input for the window corresponding to the - * window of the main input element. + * main input's window in which values are being combined. */ public abstract T sideInput(PCollectionView view); } From cf4229a6d7e1b79416a1be4e78f5c90c38dd77b0 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 14 Dec 2016 14:52:34 -0800 Subject: [PATCH 191/279] Fail to split in FileBasedSource if filePattern expands to empty. Typically, input file patterns are validated during Pipeline construction, but standard Read transforms include an option to disable validation. This is generally useful but can lead to cases where a Pipeline executes successfully with empty inputs. This changes the behavior to fail execution on empty file-based inputs even when validation is disabled. --- .../java/org/apache/beam/sdk/io/FileBasedSource.java | 6 +++++- .../org/apache/beam/sdk/io/FileBasedSourceTest.java | 12 ++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java index d835f9b9cd15..5659d5b7d154 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java @@ -331,7 +331,11 @@ public final List> splitIntoBundles( try { checkState(fileOrPatternSpec.isAccessible(), "Bundle splitting should only happen at execution time."); - for (final String file : FileBasedSource.expandFilePattern(fileOrPatternSpec.get())) { + Collection expandedFiles = + FileBasedSource.expandFilePattern(fileOrPatternSpec.get()); + checkArgument(!expandedFiles.isEmpty(), + "Unable to find any files matching %s", fileOrPatternSpec.get()); + for (final String file : expandedFiles) { futures.add(createFutureForFileSplit(file, desiredBundleSizeBytes, options, service)); } List> splitResults = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java index a065191cb317..f4b857473239 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java @@ -59,6 +59,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -73,6 +74,7 @@ public class FileBasedSourceTest { Random random = new Random(0L); @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @Rule public ExpectedException thrown = ExpectedException.none(); /** * If {@code splitHeader} is null, this is just a simple line-based reader. Otherwise, the file is @@ -417,6 +419,16 @@ public void testSplittingUsingFullThreadPool() throws Exception { assertEquals(numFiles, splits.size()); } + @Test + public void testSplittingFailsOnEmptyFileExpansion() throws Exception { + PipelineOptions options = PipelineOptionsFactory.create(); + String missingFilePath = tempFolder.newFolder().getAbsolutePath() + "/missing.txt"; + TestFileBasedSource source = new TestFileBasedSource(missingFilePath, Long.MAX_VALUE, null); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(String.format("Unable to find any files matching %s", missingFilePath)); + source.splitIntoBundles(1234, options); + } + @Test public void testFractionConsumedWhenReadingFilepattern() throws IOException { List data1 = createStringDataset(3, 1000); From bba3700aebd9dcaf88402b6845008cf7e5950cfe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 15 Dec 2016 14:34:20 +0100 Subject: [PATCH 192/279] [BEAM-932] Enable findbugs validation (and fix existing issues) --- runners/spark/pom.xml | 10 ---- .../metrics/WithNamedAggregatorsSupport.java | 4 +- .../spark/translation/WindowingHelpers.java | 3 +- .../main/resources/beam/findbugs-filter.xml | 49 ++++++++++++++++--- 4 files changed, 47 insertions(+), 19 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index e8fffa2b91e3..5a2fe87ab4cb 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -146,7 +146,6 @@ com.google.code.findbugs jsr305 - 1.3.9 com.google.guava @@ -317,15 +316,6 @@ - - - org.codehaus.mojo - findbugs-maven-plugin - - true - - - org.apache.maven.plugins maven-surefire-plugin diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java index 6932ae6ace73..5e712803224c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java @@ -121,8 +121,8 @@ public Map apply(final Map.Entry entry) { final String parentName = entry.getKey(); final Map gaugeMap = Maps.transformEntries(agg.renderAll(), toGauge()); final Map fullNameGaugeMap = Maps.newLinkedHashMap(); - for (String shortName : gaugeMap.keySet()) { - fullNameGaugeMap.put(parentName + "." + shortName, gaugeMap.get(shortName)); + for (Map.Entry gaugeEntry : gaugeMap.entrySet()) { + fullNameGaugeMap.put(parentName + "." + gaugeEntry.getKey(), gaugeEntry.getValue()); } return Maps.filterValues(fullNameGaugeMap, Predicates.notNull()); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java index ec94f3e991a2..0acff71e673b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/WindowingHelpers.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.spark.translation; +import javax.annotation.Nonnull; import org.apache.beam.sdk.util.WindowedValue; import org.apache.spark.api.java.function.Function; @@ -84,7 +85,7 @@ public WindowedValue apply(T t) { public static com.google.common.base.Function, T> unwindowValueFunction() { return new com.google.common.base.Function, T>() { @Override - public T apply(WindowedValue t) { + public T apply(@Nonnull WindowedValue t) { return t.getValue(); } }; diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index a696aebe413c..bfb498887309 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -124,6 +124,44 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -202,18 +240,17 @@ - - - - - - + + + + + From 8c00f63ca264e73c7ab7cfc760ef7748edbb6f60 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 10:20:03 -0800 Subject: [PATCH 193/279] Fix mvn command args in Apex postcommit Jenkins job --- .../job_beam_PostCommit_Java_RunnableOnService_Apex.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Apex.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Apex.groovy index 79d12bea8e69..232c94e1d5c4 100644 --- a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Apex.groovy +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Apex.groovy @@ -36,6 +36,6 @@ mavenJob('beam_PostCommit_Java_RunnableOnService_Apex') { --also-make \ --batch-mode \ --errors \ - --profile runnable-on-service-tests \ - --profile local-runnable-on-service-tests''') + --activate-profiles runnable-on-service-tests \ + --activate-profiles local-runnable-on-service-tests''') } From 519008303f9cefd3f8f4a8a7a98a9a79717f57ff Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Thu, 17 Nov 2016 10:57:41 -0800 Subject: [PATCH 194/279] [BEAM-1022] Add testing coverage for BigQuery streaming writes --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 48 +- .../sdk/io/gcp/bigquery/BigQueryServices.java | 7 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 121 ++++- .../gcp/bigquery/BigQueryTableInserter.java | 217 --------- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 456 ++++++++++++++---- .../bigquery/BigQueryServicesImplTest.java | 139 +++++- .../bigquery/BigQueryTableInserterTest.java | 245 ---------- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 50 +- 8 files changed, 655 insertions(+), 628 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java delete mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 0be8567fe5fd..28049ed27492 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.api.client.json.JsonFactory; -import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; @@ -33,6 +32,7 @@ import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatistics; import com.google.api.services.bigquery.model.JobStatus; +import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; @@ -1796,8 +1796,8 @@ public Bound withSchema(ValueProvider schema) { *

      Does not modify this object. */ public Bound withCreateDisposition(CreateDisposition createDisposition) { - return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition, - writeDisposition, validate, bigQueryServices); + return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, + createDisposition, writeDisposition, validate, bigQueryServices); } /** @@ -1806,8 +1806,8 @@ public Bound withCreateDisposition(CreateDisposition createDisposition) { *

      Does not modify this object. */ public Bound withWriteDisposition(WriteDisposition writeDisposition) { - return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition, - writeDisposition, validate, bigQueryServices); + return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, + createDisposition, writeDisposition, validate, bigQueryServices); } /** @@ -2136,7 +2136,8 @@ public TableSchema getSchema() { /** Returns the table reference, or {@code null}. */ @Nullable public ValueProvider getTable() { - return NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableRef()); + return jsonTableRef == null ? null : + NestedValueProvider.of(jsonTableRef, new JsonTableRefToTableRef()); } /** Returns {@code true} if table validation is enabled. */ @@ -2550,6 +2551,13 @@ private static void verifyTablePresence(DatasetService datasetService, TableRefe } } + /** + * Clear the cached map of created tables. Used for testing. + */ + @VisibleForTesting + static void clearCreatedTables() { + StreamingWriteFn.clearCreatedTables(); + } ///////////////////////////////////////////////////////////////////////////// /** @@ -2585,6 +2593,15 @@ private static class StreamingWriteFn this.bqServices = checkNotNull(bqServices, "bqServices"); } + /** + * Clear the cached map of created tables. Used for testing. + */ + private static void clearCreatedTables() { + synchronized (createdTables) { + createdTables.clear(); + } + } + /** Prepares a target BigQuery table. */ @StartBundle public void startBundle(Context context) { @@ -2626,20 +2643,25 @@ public void populateDisplayData(DisplayData.Builder builder) { } public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec) - throws IOException { + throws InterruptedException, IOException { TableReference tableReference = parseTableSpec(tableSpec); if (!createdTables.contains(tableSpec)) { synchronized (createdTables) { // Another thread may have succeeded in creating the table in the meanwhile, so // check again. This check isn't needed for correctness, but we add it to prevent // every thread from attempting a create and overwhelming our BigQuery quota. + DatasetService datasetService = bqServices.getDatasetService(options); if (!createdTables.contains(tableSpec)) { - TableSchema tableSchema = JSON_FACTORY.fromString( - jsonTableSchema.get(), TableSchema.class); - Bigquery client = Transport.newBigQueryClient(options).build(); - BigQueryTableInserter inserter = new BigQueryTableInserter(client, options); - inserter.getOrCreateTable(tableReference, Write.WriteDisposition.WRITE_APPEND, - Write.CreateDisposition.CREATE_IF_NEEDED, tableSchema); + Table table = datasetService.getTable( + tableReference.getProjectId(), + tableReference.getDatasetId(), + tableReference.getTableId()); + if (table == null) { + TableSchema tableSchema = JSON_FACTORY.fromString( + jsonTableSchema.get(), TableSchema.class); + datasetService.createTable( + new Table().setTableReference(tableReference).setSchema(tableSchema)); + } createdTables.add(tableSpec); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index 07dc06e24c54..8ca473d79af5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -114,11 +114,16 @@ JobStatistics dryRunQuery(String projectId, JobConfigurationQuery queryConfig) */ interface DatasetService { /** - * Gets the specified {@link Table} resource by table ID. + * Gets the specified {@link Table} resource by table ID or {@code null} if no table exists. */ Table getTable(String projectId, String datasetId, String tableId) throws InterruptedException, IOException; + /** + * Creates the specified table if it does not exist. + */ + void createTable(Table table) throws InterruptedException, IOException; + /** * Deletes the table specified by tableId from the dataset. * If the table contains data, all the data will be deleted. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 61f1a1a6a063..4eb8e7b451c4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -23,6 +23,7 @@ import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.ExponentialBackOff; import com.google.api.client.util.Sleeper; import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Dataset; @@ -53,10 +54,12 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.beam.sdk.options.BigQueryOptions; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Transport; import org.joda.time.Duration; @@ -281,7 +284,8 @@ public JobStatistics dryRunQuery(String projectId, JobConfigurationQuery queryCo "Unable to dry run query: %s, aborting after %d retries.", queryConfig, MAX_RPC_RETRIES), Sleeper.DEFAULT, - backoff).getStatistics(); + backoff, + ALWAYS_RETRY).getStatistics(); } /** @@ -400,7 +404,80 @@ public Table getTable(String projectId, String datasetId, String tableId) "Unable to get table: %s, aborting after %d retries.", tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, - backoff); + backoff, + DONT_RETRY_NOT_FOUND); + } + + /** + * Retry table creation up to 5 minutes (with exponential backoff) when this user is near the + * quota for table creation. This relatively innocuous behavior can happen when BigQueryIO is + * configured with a table spec function to use different tables for each window. + */ + private static final int RETRY_CREATE_TABLE_DURATION_MILLIS = + (int) TimeUnit.MINUTES.toMillis(5); + + /** + * {@inheritDoc} + * + *

      If a table with the same name already exists in the dataset, the function simply + * returns. In such a case, + * the existing table doesn't necessarily have the same schema as specified + * by the parameter. + * + * @throws IOException if other error than already existing table occurs. + */ + @Override + public void createTable(Table table) throws InterruptedException, IOException { + LOG.info("Trying to create BigQuery table: {}", + BigQueryIO.toTableSpec(table.getTableReference())); + BackOff backoff = + new ExponentialBackOff.Builder() + .setMaxElapsedTimeMillis(RETRY_CREATE_TABLE_DURATION_MILLIS) + .build(); + + tryCreateTable(table, backoff, Sleeper.DEFAULT); + } + + @VisibleForTesting + @Nullable + Table tryCreateTable(Table table, BackOff backoff, Sleeper sleeper) + throws IOException { + boolean retry = false; + while (true) { + try { + return client.tables().insert( + table.getTableReference().getProjectId(), + table.getTableReference().getDatasetId(), + table).execute(); + } catch (IOException e) { + ApiErrorExtractor extractor = new ApiErrorExtractor(); + if (extractor.itemAlreadyExists(e)) { + // The table already exists, nothing to return. + return null; + } else if (extractor.rateLimited(e)) { + // The request failed because we hit a temporary quota. Back off and try again. + try { + if (BackOffUtils.next(sleeper, backoff)) { + if (!retry) { + LOG.info( + "Quota limit reached when creating table {}:{}.{}, retrying up to {} minutes", + table.getTableReference().getProjectId(), + table.getTableReference().getDatasetId(), + table.getTableReference().getTableId(), + TimeUnit.MILLISECONDS.toSeconds(RETRY_CREATE_TABLE_DURATION_MILLIS) / 60.0); + retry = true; + } + continue; + } + } catch (InterruptedException e1) { + // Restore interrupted state and throw the last failure. + Thread.currentThread().interrupt(); + throw e; + } + } + throw e; + } + } } /** @@ -422,7 +499,8 @@ public void deleteTable(String projectId, String datasetId, String tableId) "Unable to delete table: %s, aborting after %d retries.", tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, - backoff); + backoff, + ALWAYS_RETRY); } @Override @@ -437,7 +515,8 @@ public boolean isTableEmpty(String projectId, String datasetId, String tableId) "Unable to list table data: %s, aborting after %d retries.", tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, - backoff); + backoff, + ALWAYS_RETRY); return dataList.getRows() == null || dataList.getRows().isEmpty(); } @@ -460,7 +539,8 @@ public Dataset getDataset(String projectId, String datasetId) "Unable to get dataset: %s, aborting after %d retries.", datasetId, MAX_RPC_RETRIES), Sleeper.DEFAULT, - backoff); + backoff, + DONT_RETRY_NOT_FOUND); } /** @@ -543,7 +623,8 @@ public void deleteDataset(String projectId, String datasetId) "Unable to delete table: %s, aborting after %d retries.", datasetId, MAX_RPC_RETRIES), Sleeper.DEFAULT, - backoff); + backoff, + ALWAYS_RETRY); } @VisibleForTesting @@ -684,8 +765,8 @@ public List call() throws IOException { public long insertAll( TableReference ref, List rowList, @Nullable List insertIdList) throws IOException, InterruptedException { - return insertAll( - ref, rowList, insertIdList, INSERT_BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT); + return insertAll( + ref, rowList, insertIdList, INSERT_BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT); } } @@ -747,12 +828,31 @@ public void close() throws IOException { } } + static final SerializableFunction DONT_RETRY_NOT_FOUND = + new SerializableFunction() { + @Override + public Boolean apply(IOException input) { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + return !errorExtractor.itemNotFound(input); + } + }; + + static final SerializableFunction ALWAYS_RETRY = + new SerializableFunction() { + @Override + public Boolean apply(IOException input) { + return true; + } + }; + + @VisibleForTesting static T executeWithRetries( AbstractGoogleClientRequest request, String errorMessage, Sleeper sleeper, - BackOff backoff) + BackOff backoff, + SerializableFunction shouldRetry) throws IOException, InterruptedException { Exception lastException = null; do { @@ -761,6 +861,9 @@ static T executeWithRetries( } catch (IOException e) { LOG.warn("Ignore the error and retry the request.", e); lastException = e; + if (!shouldRetry.apply(e)) { + break; + } } } while (nextBackOff(sleeper, backoff)); throw new IOException( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java deleted file mode 100644 index a64dc9fe9b9a..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserter.java +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.gcp.bigquery; - -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.ExponentialBackOff; -import com.google.api.client.util.Sleeper; -import com.google.api.services.bigquery.Bigquery; -import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableDataList; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.cloud.hadoop.util.ApiErrorExtractor; -import com.google.common.annotations.VisibleForTesting; -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; -import org.apache.beam.sdk.options.PipelineOptions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Inserts rows into BigQuery. - */ -class BigQueryTableInserter { - private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableInserter.class); - - private final Bigquery client; - - /** - * Constructs a new row inserter. - * - * @param client a BigQuery client - * @param options a PipelineOptions object - */ - BigQueryTableInserter(Bigquery client, PipelineOptions options) { - this.client = client; - } - - /** - * Retrieves or creates the table. - * - *

      The table is checked to conform to insertion requirements as specified - * by WriteDisposition and CreateDisposition. - * - *

      If table truncation is requested (WriteDisposition.WRITE_TRUNCATE), then - * this will re-create the table if necessary to ensure it is empty. - * - *

      If an empty table is required (WriteDisposition.WRITE_EMPTY), then this - * will fail if the table exists and is not empty. - * - *

      When constructing a table, a {@code TableSchema} must be available. If a - * schema is provided, then it will be used. If no schema is provided, but - * an existing table is being cleared (WRITE_TRUNCATE option above), then - * the existing schema will be re-used. If no schema is available, then an - * {@code IOException} is thrown. - */ - Table getOrCreateTable( - TableReference ref, - WriteDisposition writeDisposition, - CreateDisposition createDisposition, - @Nullable TableSchema schema) throws IOException { - // Check if table already exists. - Bigquery.Tables.Get get = client.tables() - .get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); - Table table = null; - try { - table = get.execute(); - } catch (IOException e) { - ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); - if (!errorExtractor.itemNotFound(e) - || createDisposition != CreateDisposition.CREATE_IF_NEEDED) { - // Rethrow. - throw e; - } - } - - // If we want an empty table, and it isn't, then delete it first. - if (table != null) { - if (writeDisposition == WriteDisposition.WRITE_APPEND) { - return table; - } - - boolean empty = isEmpty(ref); - if (empty) { - if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) { - LOG.info("Empty table found, not removing {}", BigQueryIO.toTableSpec(ref)); - } - return table; - - } else if (writeDisposition == WriteDisposition.WRITE_EMPTY) { - throw new IOException("WriteDisposition is WRITE_EMPTY, " - + "but table is not empty"); - } - - // Reuse the existing schema if none was provided. - if (schema == null) { - schema = table.getSchema(); - } - - // Delete table and fall through to re-creating it below. - LOG.info("Deleting table {}", BigQueryIO.toTableSpec(ref)); - Bigquery.Tables.Delete delete = client.tables() - .delete(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); - delete.execute(); - } - - if (schema == null) { - throw new IllegalArgumentException( - "Table schema required for new table."); - } - - // Create the table. - return tryCreateTable(ref, schema); - } - - /** - * Checks if a table is empty. - */ - private boolean isEmpty(TableReference ref) throws IOException { - Bigquery.Tabledata.List list = client.tabledata() - .list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); - list.setMaxResults(1L); - TableDataList dataList = list.execute(); - - return dataList.getRows() == null || dataList.getRows().isEmpty(); - } - - /** - * Retry table creation up to 5 minutes (with exponential backoff) when this user is near the - * quota for table creation. This relatively innocuous behavior can happen when BigQueryIO is - * configured with a table spec function to use different tables for each window. - */ - private static final int RETRY_CREATE_TABLE_DURATION_MILLIS = (int) TimeUnit.MINUTES.toMillis(5); - - /** - * Tries to create the BigQuery table. - * If a table with the same name already exists in the dataset, the table - * creation fails, and the function returns null. In such a case, - * the existing table doesn't necessarily have the same schema as specified - * by the parameter. - * - * @param schema Schema of the new BigQuery table. - * @return The newly created BigQuery table information, or null if the table - * with the same name already exists. - * @throws IOException if other error than already existing table occurs. - */ - @Nullable - private Table tryCreateTable(TableReference ref, TableSchema schema) throws IOException { - LOG.info("Trying to create BigQuery table: {}", BigQueryIO.toTableSpec(ref)); - BackOff backoff = - new ExponentialBackOff.Builder() - .setMaxElapsedTimeMillis(RETRY_CREATE_TABLE_DURATION_MILLIS) - .build(); - - Table table = new Table().setTableReference(ref).setSchema(schema); - return tryCreateTable(table, ref.getProjectId(), ref.getDatasetId(), backoff, Sleeper.DEFAULT); - } - - @VisibleForTesting - @Nullable - Table tryCreateTable( - Table table, String projectId, String datasetId, BackOff backoff, Sleeper sleeper) - throws IOException { - boolean retry = false; - while (true) { - try { - return client.tables().insert(projectId, datasetId, table).execute(); - } catch (IOException e) { - ApiErrorExtractor extractor = new ApiErrorExtractor(); - if (extractor.itemAlreadyExists(e)) { - // The table already exists, nothing to return. - return null; - } else if (extractor.rateLimited(e)) { - // The request failed because we hit a temporary quota. Back off and try again. - try { - if (BackOffUtils.next(sleeper, backoff)) { - if (!retry) { - LOG.info( - "Quota limit reached when creating table {}:{}.{}, retrying up to {} minutes", - projectId, - datasetId, - table.getTableReference().getTableId(), - TimeUnit.MILLISECONDS.toSeconds(RETRY_CREATE_TABLE_DURATION_MILLIS) / 60.0); - retry = true; - } - continue; - } - } catch (InterruptedException e1) { - // Restore interrupted state and throw the last failure. - Thread.currentThread().interrupt(); - throw e; - } - } - throw e; - } - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 54ec2bb9b4cc..b78316f7b7e7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -23,11 +23,13 @@ import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.fromJsonString; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.toJsonString; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doNothing; @@ -58,18 +60,20 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.collect.Table.Cell; import java.io.ByteArrayInputStream; import java.io.File; import java.io.FileFilter; import java.io.IOException; +import java.io.InputStream; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.io.OutputStream; import java.io.Serializable; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -85,6 +89,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -133,6 +139,9 @@ import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; @@ -146,6 +155,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; +import org.joda.time.Instant; import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; @@ -478,39 +488,81 @@ private Object[] replaceBytesWithJobs(Object obj) throws IOException { } } - /** A fake dataset service that can be serialized, for use in testReadFromTable. */ - private static class FakeDatasetService implements DatasetService, Serializable { - private com.google.common.collect.Table> tables = - HashBasedTable.create(); - - public FakeDatasetService withTable( - String projectId, String datasetId, String tableId, Table table) throws IOException { - Map dataset = tables.get(projectId, datasetId); - if (dataset == null) { - dataset = new HashMap<>(); - tables.put(projectId, datasetId, dataset); - } - dataset.put(tableId, table); + private static class TableContainer { + Table table; + List rows; + List ids; + + TableContainer(Table table) { + this.table = table; + this.rows = new ArrayList<>(); + this.ids = new ArrayList<>(); + } + + TableContainer addRow(TableRow row, String id) { + rows.add(row); + ids.add(id); return this; } + Table getTable() { + return table; + } + + List getRows() { + return rows; + } + } + + // Table information must be static, as each ParDo will get a separate instance of + // FakeDatasetServices, and they must all modify the same storage. + private static com.google.common.collect.Table> + tables = HashBasedTable.create(); + + /** A fake dataset service that can be serialized, for use in testReadFromTable. */ + private static class FakeDatasetService implements DatasetService, Serializable { + @Override public Table getTable(String projectId, String datasetId, String tableId) throws InterruptedException, IOException { - Map dataset = - checkNotNull( - tables.get(projectId, datasetId), - "Tried to get a table %s:%s.%s from %s, but no such table was set", - projectId, - datasetId, - tableId, - FakeDatasetService.class.getSimpleName()); - return checkNotNull(dataset.get(tableId), - "Tried to get a table %s:%s.%s from %s, but no such table was set", - projectId, - datasetId, - tableId, - FakeDatasetService.class.getSimpleName()); + synchronized (tables) { + Map dataset = + checkNotNull( + tables.get(projectId, datasetId), + "Tried to get a dataset %s:%s from %s, but no such dataset was set", + projectId, + datasetId, + tableId, + FakeDatasetService.class.getSimpleName()); + TableContainer tableContainer = dataset.get(tableId); + return tableContainer == null ? null : tableContainer.getTable(); + } + } + + public List getAllRows(String projectId, String datasetId, String tableId) + throws InterruptedException, IOException { + synchronized (tables) { + return getTableContainer(projectId, datasetId, tableId).getRows(); + } + } + + private TableContainer getTableContainer(String projectId, String datasetId, String tableId) + throws InterruptedException, IOException { + synchronized (tables) { + Map dataset = + checkNotNull( + tables.get(projectId, datasetId), + "Tried to get a dataset %s:%s from %s, but no such dataset was set", + projectId, + datasetId, + FakeDatasetService.class.getSimpleName()); + return checkNotNull(dataset.get(tableId), + "Tried to get a table %s:%s.%s from %s, but no such table was set", + projectId, + datasetId, + tableId, + FakeDatasetService.class.getSimpleName()); + } } @Override @@ -519,6 +571,26 @@ public void deleteTable(String projectId, String datasetId, String tableId) throw new UnsupportedOperationException("Unsupported"); } + + @Override + public void createTable(Table table) throws IOException { + TableReference tableReference = table.getTableReference(); + synchronized (tables) { + Map dataset = + checkNotNull( + tables.get(tableReference.getProjectId(), tableReference.getDatasetId()), + "Tried to get a dataset %s:%s from %s, but no such table was set", + tableReference.getProjectId(), + tableReference.getDatasetId(), + FakeDatasetService.class.getSimpleName()); + TableContainer tableContainer = dataset.get(tableReference.getTableId()); + if (tableContainer == null) { + tableContainer = new TableContainer(table); + dataset.put(tableReference.getTableId(), tableContainer); + } + } + } + @Override public boolean isTableEmpty(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { @@ -536,7 +608,13 @@ public Dataset getDataset( public void createDataset( String projectId, String datasetId, String location, String description) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); + synchronized (tables) { + Map dataset = tables.get(projectId, datasetId); + if (dataset == null) { + dataset = new HashMap<>(); + tables.put(projectId, datasetId, dataset); + } + } } @Override @@ -549,55 +627,18 @@ public void deleteDataset(String projectId, String datasetId) public long insertAll( TableReference ref, List rowList, @Nullable List insertIdList) throws IOException, InterruptedException { - throw new UnsupportedOperationException("Unsupported"); - } - - ////////////////////////////////// SERIALIZATION METHODS //////////////////////////////////// - private void writeObject(ObjectOutputStream out) throws IOException { - out.writeObject(replaceTablesWithBytes(this.tables)); - } - - private com.google.common.collect.Table> - replaceTablesWithBytes( - com.google.common.collect.Table> toCopy) - throws IOException { - com.google.common.collect.Table> copy = - HashBasedTable.create(); - for (Cell> cell : toCopy.cellSet()) { - HashMap dataset = new HashMap<>(); - copy.put(cell.getRowKey(), cell.getColumnKey(), dataset); - for (Map.Entry dsTables : cell.getValue().entrySet()) { - dataset.put( - dsTables.getKey(), Transport.getJsonFactory().toByteArray(dsTables.getValue())); + synchronized (tables) { + assertEquals(rowList.size(), insertIdList.size()); + + long dataSize = 0; + TableContainer tableContainer = getTableContainer( + ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + for (int i = 0; i < rowList.size(); ++i) { + tableContainer.addRow(rowList.get(i), insertIdList.get(i)); + dataSize += rowList.get(i).toString().length(); } + return dataSize; } - return copy; - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - com.google.common.collect.Table> tablesTable = - (com.google.common.collect.Table>) in.readObject(); - this.tables = replaceBytesWithTables(tablesTable); - } - - private com.google.common.collect.Table> - replaceBytesWithTables( - com.google.common.collect.Table> tablesTable) - throws IOException { - com.google.common.collect.Table> copy = - HashBasedTable.create(); - for (Cell> cell : tablesTable.cellSet()) { - HashMap dataset = new HashMap<>(); - copy.put(cell.getRowKey(), cell.getColumnKey(), dataset); - for (Map.Entry dsTables : cell.getValue().entrySet()) { - Table table = - Transport.getJsonFactory() - .createJsonParser(new ByteArrayInputStream(dsTables.getValue())) - .parse(Table.class); - dataset.put(dsTables.getKey(), table); - } - } - return copy; } } @@ -658,6 +699,8 @@ private void checkWriteObjectWithValidate( @Before public void setUp() throws IOException { MockitoAnnotations.initMocks(this); + tables = HashBasedTable.create(); + BigQueryIO.clearCreatedTables(); } @Test @@ -716,8 +759,11 @@ public void testValidateReadSetsDefaultProject() throws Exception { bqOptions.setProject(projectId); bqOptions.setTempLocation("gs://testbucket/testdir"); - FakeDatasetService fakeDatasetService = - new FakeDatasetService().withTable(projectId, datasetId, tableId, null); + FakeDatasetService fakeDatasetService = new FakeDatasetService(); + fakeDatasetService.createDataset(projectId, datasetId, "", ""); + TableReference tableReference = + new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId); + fakeDatasetService.createTable(new Table().setTableReference(tableReference)); FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService()) @@ -765,7 +811,7 @@ public void testBuildSourceWithTableAndQuery() { p.apply("ReadMyTable", BigQueryIO.Read .from("foo.com:project:somedataset.sometable") - .fromQuery("query")); + .fromQuery("query")); p.run(); } @@ -829,7 +875,7 @@ public void testBuildSourceWithTableAndSqlDialect() { @Test @Category(NeedsRunner.class) - public void testReadFromTable() throws IOException { + public void testReadFromTable() throws IOException, InterruptedException { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); @@ -850,10 +896,15 @@ public void testReadFromTable() throws IOException { ImmutableList.of( new TableFieldSchema().setName("name").setType("STRING"), new TableFieldSchema().setName("number").setType("INTEGER")))); + sometable.setTableReference( + new TableReference() + .setProjectId("non-executing-project") + .setDatasetId("somedataset") + .setTableId("sometable")); sometable.setNumBytes(1024L * 1024L); - FakeDatasetService fakeDatasetService = - new FakeDatasetService() - .withTable("non-executing-project", "somedataset", "sometable", sometable); + FakeDatasetService fakeDatasetService = new FakeDatasetService(); + fakeDatasetService.createDataset("non-executing-project", "somedataset", "", ""); + fakeDatasetService.createTable(sometable); SerializableFunction schemaGenerator = new SerializableFunction() { @Override @@ -943,6 +994,216 @@ public void testWrite() throws Exception { testNumFiles(tempDir, 0); } + @Test + @Category(NeedsRunner.class) + public void testStreamingWrite() throws Exception { + BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); + bqOptions.setProject("defaultProject"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + + FakeDatasetService datasetService = new FakeDatasetService(); + datasetService.createDataset("project-id", "dataset-id", "", ""); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() + .withDatasetService(datasetService); + + Pipeline p = TestPipeline.create(bqOptions); + p.apply(Create.of( + new TableRow().set("name", "a").set("number", 1), + new TableRow().set("name", "b").set("number", 2), + new TableRow().set("name", "c").set("number", 3), + new TableRow().set("name", "d").set("number", 4)) + .withCoder(TableRowJsonCoder.of())) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED) + .apply(BigQueryIO.Write.to("project-id:dataset-id.table-id") + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withSchema(new TableSchema().setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER")))) + .withTestServices(fakeBqServices) + .withoutValidation()); + p.run(); + + + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id"), + containsInAnyOrder( + new TableRow().set("name", "a").set("number", 1), + new TableRow().set("name", "b").set("number", 2), + new TableRow().set("name", "c").set("number", 3), + new TableRow().set("name", "d").set("number", 4))); + } + + /** + * A generic window function that allows partitioning data into windows by a string value. + * + *

      Logically, creates multiple global windows, and the user provides a function that + * decides which global window a value should go into. + */ + private static class PartitionedGlobalWindows extends + + NonMergingWindowFn { + private SerializableFunction extractPartition; + + public PartitionedGlobalWindows(SerializableFunction extractPartition) { + this.extractPartition = extractPartition; + } + + @Override + public Collection assignWindows(AssignContext c) { + return Collections.singletonList(new PartitionedGlobalWindow( + extractPartition.apply(c.element()))); + } + + @Override + public boolean isCompatible(WindowFn o) { + return o instanceof PartitionedGlobalWindows; + } + + @Override + public Coder windowCoder() { + return new PartitionedGlobalWindowCoder(); + } + + @Override + public PartitionedGlobalWindow getSideInputWindow(BoundedWindow window) { + throw new UnsupportedOperationException( + "PartitionedGlobalWindows is not allowed in side inputs"); + } + + @Override + public Instant getOutputTime(Instant inputTimestamp, PartitionedGlobalWindow window) { + return inputTimestamp; + } + } + + /** + * Custom Window object that encodes a String value. + */ + private static class PartitionedGlobalWindow extends BoundedWindow { + String value; + + public PartitionedGlobalWindow(String value) { + this.value = value; + } + + @Override + public Instant maxTimestamp() { + return GlobalWindow.INSTANCE.maxTimestamp(); + } + + // The following methods are only needed due to BEAM-1022. Once this issue is fixed, we will + // no longer need these. + @Override + public boolean equals(Object other) { + if (other instanceof PartitionedGlobalWindow) { + return value.equals(((PartitionedGlobalWindow) other).value); + } + return false; + } + + @Override + public int hashCode() { + return value.hashCode(); + } + } + + /** + * Coder for @link{PartitionedGlobalWindow}. + */ + private static class PartitionedGlobalWindowCoder extends AtomicCoder { + @Override + public void encode(PartitionedGlobalWindow window, OutputStream outStream, Context context) + throws IOException, CoderException { + StringUtf8Coder.of().encode(window.value, outStream, context); + } + + @Override + public PartitionedGlobalWindow decode(InputStream inStream, Context context) + throws IOException, CoderException { + return new PartitionedGlobalWindow(StringUtf8Coder.of().decode(inStream, context)); + } + } + + @Test + @Category(NeedsRunner.class) + public void testStreamingWriteWithWindowFn() throws Exception { + BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); + bqOptions.setProject("defaultProject"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + + FakeDatasetService datasetService = new FakeDatasetService(); + datasetService.createDataset("project-id", "dataset-id", "", ""); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() + .withDatasetService(datasetService); + + List inserts = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + inserts.add(new TableRow().set("name", "number" + i).set("number", i)); + } + + // Create a windowing strategy that puts the input into five different windows depending on + // record value. + WindowFn window = new PartitionedGlobalWindows( + new SerializableFunction() { + @Override + public String apply(TableRow value) { + try { + int intValue = (Integer) value.get("number") % 5; + return Integer.toString(intValue); + } catch (NumberFormatException e) { + fail(e.toString()); + } + return value.toString(); + } + } + ); + + SerializableFunction tableFunction = + new SerializableFunction() { + @Override + public String apply(BoundedWindow input) { + return "project-id:dataset-id.table-id-" + ((PartitionedGlobalWindow) input).value; + } + }; + + Pipeline p = TestPipeline.create(bqOptions); + p.apply(Create.of(inserts).withCoder(TableRowJsonCoder.of())) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED) + .apply(Window.into(window)) + .apply(BigQueryIO.Write + .to(tableFunction) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withSchema(new TableSchema().setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER")))) + .withTestServices(fakeBqServices) + .withoutValidation()); + p.run(); + + + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-0"), + containsInAnyOrder( + new TableRow().set("name", "number0").set("number", 0), + new TableRow().set("name", "number5").set("number", 5))); + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-1"), + containsInAnyOrder( + new TableRow().set("name", "number1").set("number", 1), + new TableRow().set("name", "number6").set("number", 6))); + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-2"), + containsInAnyOrder( + new TableRow().set("name", "number2").set("number", 2), + new TableRow().set("name", "number7").set("number", 7))); + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-3"), + containsInAnyOrder( + new TableRow().set("name", "number3").set("number", 3), + new TableRow().set("name", "number8").set("number", 8))); + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-4"), + containsInAnyOrder( + new TableRow().set("name", "number4").set("number", 4), + new TableRow().set("name", "number9").set("number", 9))); + } + @Test @Category(NeedsRunner.class) public void testWriteUnknown() throws Exception { @@ -1031,7 +1292,8 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte @Test public void testBuildWrite() { - BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable"); + BigQueryIO.Write.Bound bound = + BigQueryIO.Write.to("foo.com:project:somedataset.sometable"); checkWriteObject( bound, "foo.com:project", "somedataset", "sometable", null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); @@ -1980,32 +2242,42 @@ public interface RuntimeTestOptions extends PipelineOptions { } @Test - public void testRuntimeOptionsNotCalledInApplyInputTable() { + public void testRuntimeOptionsNotCalledInApplyInputTable() throws IOException { RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - bqOptions.setTempLocation("gs://testbucket/testdir"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() + .withJobService(new FakeJobService()); Pipeline pipeline = TestPipeline.create(options); pipeline - .apply(BigQueryIO.Read.from(options.getInputTable()).withoutValidation()) - .apply(BigQueryIO.Write + .apply(BigQueryIO.Read + .from(options.getInputTable()).withoutValidation() + .withTestServices(fakeBqServices)) + .apply(BigQueryIO.Write .to(options.getOutputTable()) .withSchema(NestedValueProvider.of( options.getOutputSchema(), new JsonSchemaToTableSchema())) + .withTestServices(fakeBqServices) .withoutValidation()); } @Test - public void testRuntimeOptionsNotCalledInApplyInputQuery() { + public void testRuntimeOptionsNotCalledInApplyInputQuery() throws IOException { RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - bqOptions.setTempLocation("gs://testbucket/testdir"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() + .withJobService(new FakeJobService()); Pipeline pipeline = TestPipeline.create(options); pipeline - .apply(BigQueryIO.Read.fromQuery(options.getInputQuery()).withoutValidation()) - .apply(BigQueryIO.Write + .apply(BigQueryIO.Read + .fromQuery(options.getInputQuery()).withoutValidation() + .withTestServices(fakeBqServices)) + .apply(BigQueryIO.Write .to(options.getOutputTable()) .withSchema(NestedValueProvider.of( options.getOutputSchema(), new JsonSchemaToTableSchema())) + .withTestServices(fakeBqServices) .withoutValidation()); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index 0e76660ec428..10ed8bd12c9d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -17,9 +17,11 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static com.google.common.base.Verify.verifyNotNull; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; import static org.mockito.Mockito.times; @@ -47,9 +49,12 @@ import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableDataInsertAllResponse; import com.google.api.services.bigquery.model.TableDataInsertAllResponse.InsertErrors; +import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.hadoop.util.ApiErrorExtractor; +import com.google.cloud.hadoop.util.RetryBoundedBackOff; import com.google.common.collect.ImmutableList; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -64,6 +69,7 @@ import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; +import org.joda.time.Duration; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -325,7 +331,8 @@ public void testExecuteWithRetries() throws IOException, InterruptedException { bigquery.tables().get("projectId", "datasetId", "tableId"), "Failed to get table.", Sleeper.DEFAULT, - BackOff.STOP_BACKOFF); + BackOff.STOP_BACKOFF, + BigQueryServicesImpl.ALWAYS_RETRY); assertEquals(testTable, table); verify(response, times(1)).getStatusCode(); @@ -358,6 +365,11 @@ public void testInsertRetry() throws Exception { verify(response, times(2)).getContentType(); expectedLogs.verifyInfo("BigQuery insertAll exceeded rate limit, retrying"); } + // A BackOff that makes a total of 4 attempts + private static final FluentBackoff TEST_BACKOFF = FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(1)) + .withExponent(1) + .withMaxRetries(3); /** * Tests that {@link DatasetServiceImpl#insertAll} retries selected rows on failure. @@ -371,7 +383,8 @@ public void testInsertRetrySelectRows() throws Exception { List insertIds = ImmutableList.of("a", "b"); final TableDataInsertAllResponse bFailed = new TableDataInsertAllResponse() - .setInsertErrors(ImmutableList.of(new InsertErrors().setIndex(1L))); + .setInsertErrors(ImmutableList.of( + new InsertErrors().setIndex(1L).setErrors(ImmutableList.of(new ErrorProto())))); final TableDataInsertAllResponse allRowsSucceeded = new TableDataInsertAllResponse(); @@ -389,9 +402,6 @@ public void testInsertRetrySelectRows() throws Exception { expectedLogs.verifyInfo("Retrying 1 failed inserts to BigQuery"); } - // A BackOff that makes a total of 4 attempts - private static final FluentBackoff TEST_BACKOFF = FluentBackoff.DEFAULT.withMaxRetries(3); - /** * Tests that {@link DatasetServiceImpl#insertAll} fails gracefully when persistent issues. */ @@ -490,9 +500,128 @@ private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, GoogleJsonError error = new GoogleJsonError(); error.setErrors(ImmutableList.of(info)); error.setCode(status); + error.setMessage(reason); // The actual JSON response is an error container. GoogleJsonErrorContainer container = new GoogleJsonErrorContainer(); container.setError(error); return container; } + + @Test + public void testCreateTableSucceeds() throws IOException { + TableReference ref = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + Table testTable = new Table().setTableReference(ref); + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); + when(response.getStatusCode()).thenReturn(200); + when(response.getContent()).thenReturn(toStream(testTable)); + + BigQueryServicesImpl.DatasetServiceImpl services = + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + Table ret = + services.tryCreateTable( + testTable, + new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF), + Sleeper.DEFAULT); + assertEquals(testTable, ret); + verify(response, times(1)).getStatusCode(); + verify(response, times(1)).getContent(); + verify(response, times(1)).getContentType(); + } + + /** + * Tests that {@link BigQueryServicesImpl} does not retry non-rate-limited attempts. + */ + @Test + public void testCreateTableDoesNotRetry() throws IOException { + TableReference ref = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + Table testTable = new Table().setTableReference(ref); + // First response is 403 not-rate-limited, second response has valid payload but should not + // be invoked. + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); + when(response.getStatusCode()).thenReturn(403).thenReturn(200); + when(response.getContent()) + .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403))) + .thenReturn(toStream(testTable)); + + thrown.expect(GoogleJsonResponseException.class); + thrown.expectMessage("actually forbidden"); + + BigQueryServicesImpl.DatasetServiceImpl services = + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + try { + services.tryCreateTable( + testTable, + new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF), + Sleeper.DEFAULT); + fail(); + } catch (IOException e) { + verify(response, times(1)).getStatusCode(); + verify(response, times(1)).getContent(); + verify(response, times(1)).getContentType(); + throw e; + } + } + + /** + * Tests that table creation succeeds when the table already exists. + */ + @Test + public void testCreateTableSucceedsAlreadyExists() throws IOException { + TableReference ref = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + TableSchema schema = new TableSchema().setFields(ImmutableList.of( + new TableFieldSchema().setName("column1").setType("String"), + new TableFieldSchema().setName("column2").setType("Integer"))); + Table testTable = new Table().setTableReference(ref).setSchema(schema); + + when(response.getStatusCode()).thenReturn(409); // 409 means already exists + + BigQueryServicesImpl.DatasetServiceImpl services = + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + Table ret = + services.tryCreateTable( + testTable, + new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF), + Sleeper.DEFAULT); + + assertNull(ret); + verify(response, times(1)).getStatusCode(); + verify(response, times(1)).getContent(); + verify(response, times(1)).getContentType(); + } + + /** + * Tests that {@link BigQueryServicesImpl} retries quota rate limited attempts. + */ + @Test + public void testCreateTableRetry() throws IOException { + TableReference ref = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + Table testTable = new Table().setTableReference(ref); + + // First response is 403 rate limited, second response has valid payload. + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); + when(response.getStatusCode()).thenReturn(403).thenReturn(200); + when(response.getContent()) + .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403))) + .thenReturn(toStream(testTable)); + + BigQueryServicesImpl.DatasetServiceImpl services = + new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + Table ret = + services.tryCreateTable( + testTable, + new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF), + Sleeper.DEFAULT); + assertEquals(testTable, ret); + verify(response, times(2)).getStatusCode(); + verify(response, times(2)).getContent(); + verify(response, times(2)).getContentType(); + verifyNotNull(ret.getTableReference()); + expectedLogs.verifyInfo( + "Quota limit reached when creating table project:dataset.table, " + + "retrying up to 5.0 minutes"); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java deleted file mode 100644 index fb79c74215d2..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableInserterTest.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.gcp.bigquery; - -import static com.google.common.base.Verify.verifyNotNull; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -import com.google.api.client.googleapis.json.GoogleJsonError; -import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; -import com.google.api.client.googleapis.json.GoogleJsonErrorContainer; -import com.google.api.client.googleapis.json.GoogleJsonResponseException; -import com.google.api.client.http.LowLevelHttpResponse; -import com.google.api.client.json.GenericJson; -import com.google.api.client.json.Json; -import com.google.api.client.json.jackson2.JacksonFactory; -import com.google.api.client.testing.http.MockHttpTransport; -import com.google.api.client.testing.http.MockLowLevelHttpRequest; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.Sleeper; -import com.google.api.services.bigquery.Bigquery; -import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableReference; -import com.google.cloud.hadoop.util.RetryBoundedBackOff; -import com.google.common.collect.ImmutableList; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.util.Transport; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -/** - * Tests of {@link BigQueryTableInserter}. - */ -@RunWith(JUnit4.class) -public class BigQueryTableInserterTest { - @Rule public ExpectedException thrown = ExpectedException.none(); - @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryTableInserter.class); - @Mock private LowLevelHttpResponse response; - private Bigquery bigquery; - private PipelineOptions options; - - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - - // A mock transport that lets us mock the API responses. - MockHttpTransport transport = - new MockHttpTransport.Builder() - .setLowLevelHttpRequest( - new MockLowLevelHttpRequest() { - @Override - public LowLevelHttpResponse execute() throws IOException { - return response; - } - }) - .build(); - - // A sample BigQuery API client that uses default JsonFactory and RetryHttpInitializer. - bigquery = - new Bigquery.Builder( - transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()) - .build(); - - options = PipelineOptionsFactory.create(); - } - - @After - public void tearDown() throws IOException { - // These three interactions happen for every request in the normal response parsing. - verify(response, atLeastOnce()).getContentEncoding(); - verify(response, atLeastOnce()).getHeaderCount(); - verify(response, atLeastOnce()).getReasonPhrase(); - verifyNoMoreInteractions(response); - } - - /** A helper to wrap a {@link GenericJson} object in a content stream. */ - private static InputStream toStream(GenericJson content) throws IOException { - return new ByteArrayInputStream(JacksonFactory.getDefaultInstance().toByteArray(content)); - } - - /** A helper that generates the error JSON payload that Google APIs produce. */ - private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, int status) { - ErrorInfo info = new ErrorInfo(); - info.setReason(reason); - info.setDomain("global"); - // GoogleJsonError contains one or more ErrorInfo objects; our utiities read the first one. - GoogleJsonError error = new GoogleJsonError(); - error.setErrors(ImmutableList.of(info)); - error.setCode(status); - // The actual JSON response is an error container. - GoogleJsonErrorContainer container = new GoogleJsonErrorContainer(); - container.setError(error); - return container; - } - - /** - * Tests that {@link BigQueryTableInserter} succeeds on the first try. - */ - @Test - public void testCreateTableSucceeds() throws IOException { - Table testTable = new Table().setDescription("a table"); - - when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); - when(response.getStatusCode()).thenReturn(200); - when(response.getContent()).thenReturn(toStream(testTable)); - - BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options); - Table ret = - inserter.tryCreateTable( - new Table(), - "project", - "dataset", - new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF), - Sleeper.DEFAULT); - assertEquals(testTable, ret); - verify(response, times(1)).getStatusCode(); - verify(response, times(1)).getContent(); - verify(response, times(1)).getContentType(); - } - - /** - * Tests that {@link BigQueryTableInserter} succeeds when the table already exists. - */ - @Test - public void testCreateTableSucceedsAlreadyExists() throws IOException { - when(response.getStatusCode()).thenReturn(409); // 409 means already exists - - BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options); - Table ret = - inserter.tryCreateTable( - new Table(), - "project", - "dataset", - new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF), - Sleeper.DEFAULT); - - assertNull(ret); - verify(response, times(1)).getStatusCode(); - verify(response, times(1)).getContent(); - verify(response, times(1)).getContentType(); - } - - /** - * Tests that {@link BigQueryTableInserter} retries quota rate limited attempts. - */ - @Test - public void testCreateTableRetry() throws IOException { - TableReference ref = - new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); - Table testTable = new Table().setTableReference(ref); - - // First response is 403 rate limited, second response has valid payload. - when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); - when(response.getStatusCode()).thenReturn(403).thenReturn(200); - when(response.getContent()) - .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403))) - .thenReturn(toStream(testTable)); - - BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options); - Table ret = - inserter.tryCreateTable( - testTable, - "project", - "dataset", - new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF), - Sleeper.DEFAULT); - assertEquals(testTable, ret); - verify(response, times(2)).getStatusCode(); - verify(response, times(2)).getContent(); - verify(response, times(2)).getContentType(); - verifyNotNull(ret.getTableReference()); - expectedLogs.verifyInfo( - "Quota limit reached when creating table project:dataset.table, " - + "retrying up to 5.0 minutes"); - } - - /** - * Tests that {@link BigQueryTableInserter} does not retry non-rate-limited attempts. - */ - @Test - public void testCreateTableDoesNotRetry() throws IOException { - Table testTable = new Table().setDescription("a table"); - - // First response is 403 not-rate-limited, second response has valid payload but should not - // be invoked. - when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); - when(response.getStatusCode()).thenReturn(403).thenReturn(200); - when(response.getContent()) - .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403))) - .thenReturn(toStream(testTable)); - - thrown.expect(GoogleJsonResponseException.class); - thrown.expectMessage("actually forbidden"); - - BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options); - try { - inserter.tryCreateTable( - new Table(), - "project", - "dataset", - new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF), - Sleeper.DEFAULT); - fail(); - } catch (IOException e) { - verify(response, times(1)).getStatusCode(); - verify(response, times(1)).getContent(); - verify(response, times(1)).getContentType(); - throw e; - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java index e539b33b0ee0..81302383abbf 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java @@ -361,60 +361,18 @@ public void testReadOpenFailure() throws IOException, InterruptedException { } @Test - public void testWriteAppend() throws IOException { - onTableGet(basicTableSchema()); - - TableReference ref = BigQueryIO - .parseTableSpec("project:dataset.table"); - - BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options); - - inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_APPEND, - BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null); - - verifyTableGet(); - } - - @Test - public void testWriteEmpty() throws IOException { + public void testTableGet() throws InterruptedException, IOException { onTableGet(basicTableSchema()); TableDataList dataList = new TableDataList().setTotalRows(0L); onTableList(dataList); - TableReference ref = BigQueryIO - .parseTableSpec("project:dataset.table"); - - BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options); + BigQueryServicesImpl.DatasetServiceImpl services = + new BigQueryServicesImpl.DatasetServiceImpl(mockClient, options); - inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY, - BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null); + services.getTable("project", "dataset", "table"); verifyTableGet(); - verifyTabledataList(); - } - - @Test - public void testWriteEmptyFail() throws IOException { - thrown.expect(IOException.class); - - onTableGet(basicTableSchema()); - - TableDataList dataList = rawDataList(rawRow("Arthur", 42)); - onTableList(dataList); - - TableReference ref = BigQueryIO - .parseTableSpec("project:dataset.table"); - - BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options); - - try { - inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY, - BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null); - } finally { - verifyTableGet(); - verifyTabledataList(); - } } @Test From f5f329eee4e4a446dafe15b1c42a8f0972360fbc Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 16:17:46 -0800 Subject: [PATCH 195/279] Converts all easy OldDoFns to DoFn --- .../apache/beam/runners/apex/ApexRunner.java | 11 ++---- .../FlattenPCollectionTranslatorTest.java | 15 +++---- .../translation/GroupByKeyTranslatorTest.java | 21 ++++------ .../translation/ParDoBoundTranslatorTest.java | 39 ++++++++++--------- .../ReadUnboundTranslatorTest.java | 15 +++---- .../beam/runners/flink/FlinkRunner.java | 10 ++--- .../runners/flink/PipelineOptionsTest.java | 11 +++--- .../flink/streaming/DoFnOperatorTest.java | 19 ++++----- .../flink/streaming/GroupByNullKeyTest.java | 18 ++++----- .../streaming/TopWikipediaSessionsITCase.java | 10 ++--- 10 files changed, 75 insertions(+), 94 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java index 899efa3cee37..e5bde46274d5 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -22,13 +22,11 @@ import com.datatorrent.api.DAG; import com.datatorrent.api.StreamingApplication; import com.google.common.base.Throwables; - import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicReference; - import org.apache.apex.api.EmbeddedAppLauncher; import org.apache.apex.api.Launcher; import org.apache.apex.api.Launcher.AppHandle; @@ -45,7 +43,6 @@ 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.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -245,10 +242,10 @@ public PCollectionView expand(PCollection> input) { } } - private static class WrapAsList extends OldDoFn> { - @Override + private static class WrapAsList extends DoFn> { + @ProcessElement public void processElement(ProcessContext c) { - c.output(Arrays.asList(c.element())); + c.output(Collections.singletonList(c.element())); } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java index 6b62a58537a6..f5abc34a6eda 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslatorTest.java @@ -19,12 +19,11 @@ package org.apache.beam.runners.apex.translation; import com.google.common.collect.Sets; - import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Set; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.ApexRunnerResult; @@ -32,8 +31,8 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; 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.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -83,14 +82,10 @@ public void test() throws Exception { Assert.assertEquals(expected, Sets.newHashSet(EmbeddedCollector.RESULTS)); } - @SuppressWarnings("serial") - private static class EmbeddedCollector extends OldDoFn { - protected static final ArrayList RESULTS = new ArrayList<>(); - - public EmbeddedCollector() { - } + private static class EmbeddedCollector extends DoFn { + private static final List RESULTS = Collections.synchronizedList(new ArrayList<>()); - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { RESULTS.add(c.element()); } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/GroupByKeyTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/GroupByKeyTranslatorTest.java index d627cd9b4523..96963a0f6073 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/GroupByKeyTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/GroupByKeyTranslatorTest.java @@ -20,7 +20,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import java.io.IOException; import java.io.Serializable; import java.util.Collections; @@ -28,9 +27,8 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; - +import java.util.Set; import javax.annotation.Nullable; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.ApexRunnerResult; @@ -42,7 +40,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; @@ -106,22 +104,17 @@ public void test() throws Exception { } - @SuppressWarnings("serial") - private static class EmbeddedCollector extends OldDoFn { - protected static final HashSet RESULTS = new HashSet<>(); - - public EmbeddedCollector() { - } + private static class EmbeddedCollector extends DoFn { + private static final Set RESULTS = Collections.synchronizedSet(new HashSet<>()); - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { RESULTS.add(c.element()); } } - private static class KeyedByTimestamp extends OldDoFn> { - - @Override + private static class KeyedByTimestamp extends DoFn> { + @ProcessElement public void processElement(ProcessContext c) throws Exception { c.output(KV.of(c.timestamp(), c.element())); } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java index 2e8615207aaf..28b2ec95bb9c 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java @@ -26,14 +26,13 @@ import com.datatorrent.lib.util.KryoCloneUtils; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.regex.Pattern; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.ApexRunnerResult; @@ -49,7 +48,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.View; @@ -113,8 +113,7 @@ public void test() throws Exception { Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.RESULTS); } - @SuppressWarnings("serial") - private static class Add extends OldDoFn { + private static class Add extends DoFn { private Integer number; private PCollectionView sideInputView; @@ -126,7 +125,7 @@ private Add(PCollectionView sideInputView) { this.sideInputView = sideInputView; } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { if (sideInputView != null) { number = c.sideInput(sideInputView); @@ -135,15 +134,14 @@ public void processElement(ProcessContext c) throws Exception { } } - private static class EmbeddedCollector extends OldDoFn { - private static final long serialVersionUID = 1L; - protected static final HashSet RESULTS = new HashSet<>(); + private static class EmbeddedCollector extends DoFn { + private static final Set RESULTS = Collections.synchronizedSet(new HashSet<>()); public EmbeddedCollector() { RESULTS.clear(); } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { RESULTS.add(c.element()); } @@ -207,13 +205,16 @@ public void testSerialization() throws Exception { PCollectionView singletonView = pipeline.apply(Create.of(1)) .apply(Sum.integersGlobally().asSingletonView()); - ApexParDoOperator operator = new ApexParDoOperator<>(options, - new Add(singletonView), new TupleTag(), TupleTagList.empty().getAll(), - WindowingStrategy.globalDefault(), - Collections.>singletonList(singletonView), - coder, - new ApexStateInternals.ApexStateInternalsFactory() - ); + ApexParDoOperator operator = + new ApexParDoOperator<>( + options, + DoFnAdapters.toOldDoFn(new Add(singletonView)), + new TupleTag(), + TupleTagList.empty().getAll(), + WindowingStrategy.globalDefault(), + Collections.>singletonList(singletonView), + coder, + new ApexStateInternals.ApexStateInternalsFactory()); operator.setup(null); operator.beginWindow(0); WindowedValue wv1 = WindowedValue.valueInGlobalWindow(1); @@ -303,7 +304,7 @@ public void testMultiOutputParDoWithSideInputs() throws Exception { Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.RESULTS); } - private static class TestMultiOutputWithSideInputsFn extends OldDoFn { + private static class TestMultiOutputWithSideInputsFn extends DoFn { private static final long serialVersionUID = 1L; final List> sideInputViews = new ArrayList<>(); @@ -315,7 +316,7 @@ public TestMultiOutputWithSideInputsFn(List> sideInputV this.sideOutputTupleTags.addAll(sideOutputTupleTags); } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { outputToAllWithSideInputs(c, "processing: " + c.element()); } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ReadUnboundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ReadUnboundTranslatorTest.java index 96ba6638926b..8e44bab18b0e 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ReadUnboundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ReadUnboundTranslatorTest.java @@ -24,11 +24,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.Range; import com.google.common.collect.Sets; - +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.ApexRunnerResult; @@ -39,7 +38,7 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.junit.Assert; import org.junit.Test; @@ -113,14 +112,10 @@ public void testReadBounded() throws Exception { Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.RESULTS); } - @SuppressWarnings("serial") - private static class EmbeddedCollector extends OldDoFn { - protected static final HashSet RESULTS = new HashSet<>(); - - public EmbeddedCollector() { - } + private static class EmbeddedCollector extends DoFn { + private static final Set RESULTS = Collections.synchronizedSet(new HashSet<>()); - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { RESULTS.add(c.element()); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index 7c1284b33bac..5f92378f5d5c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -24,7 +24,7 @@ import java.net.URL; import java.net.URLClassLoader; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -42,7 +42,7 @@ import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -440,10 +440,10 @@ protected String getKindString() { } } - private static class WrapAsList extends OldDoFn> { - @Override + private static class WrapAsList extends DoFn> { + @ProcessElement public void processElement(ProcessContext c) { - c.output(Arrays.asList(c.element())); + c.output(Collections.singletonList(c.element())); } } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index 3c30fed923ba..e44a705ba750 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -29,7 +29,8 @@ import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; @@ -98,7 +99,7 @@ public void testNonNull() { @Test(expected = Exception.class) public void parDoBaseClassPipelineOptionsNullTest() { DoFnOperator doFnOperator = new DoFnOperator<>( - new TestDoFn(), + DoFnAdapters.toOldDoFn(new TestDoFn()), TypeInformation.of(new TypeHint>() {}), new TupleTag<>("main-output"), Collections.>emptyList(), @@ -117,7 +118,7 @@ public void parDoBaseClassPipelineOptionsNullTest() { public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { DoFnOperator doFnOperator = new DoFnOperator<>( - new TestDoFn(), + DoFnAdapters.toOldDoFn(new TestDoFn()), TypeInformation.of(new TypeHint>() {}), new TupleTag<>("main-output"), Collections.>emptyList(), @@ -151,9 +152,9 @@ public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { } - private static class TestDoFn extends OldDoFn { + private static class TestDoFn extends DoFn { - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { Assert.assertNotNull(c.getPipelineOptions()); Assert.assertEquals( diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index 913fb8bbd1a1..65e244a9e7c6 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - import java.util.Collections; import java.util.HashMap; import javax.annotation.Nullable; @@ -35,6 +34,8 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PCollectionViewTesting; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -96,7 +97,7 @@ public void testSingleOutput() throws Exception { TupleTag outputTag = new TupleTag<>("main-output"); DoFnOperator doFnOperator = new DoFnOperator<>( - new IdentityDoFn(), + DoFnAdapters.toOldDoFn(new IdentityDoFn()), coderTypeInfo, outputTag, Collections.>emptyList(), @@ -140,7 +141,7 @@ public void testMultiOutputOutput() throws Exception { .build(); DoFnOperator doFnOperator = new DoFnOperator<>( - new MultiOutputDoFn(sideOutput1, sideOutput2), + DoFnAdapters.toOldDoFn(new MultiOutputDoFn(sideOutput1, sideOutput2)), coderTypeInfo, mainOutput, ImmutableList.>of(sideOutput1, sideOutput2), @@ -200,7 +201,7 @@ public void testSideInputs() throws Exception { .build(); DoFnOperator doFnOperator = new DoFnOperator<>( - new IdentityDoFn(), + DoFnAdapters.toOldDoFn(new IdentityDoFn()), coderTypeInfo, outputTag, Collections.>emptyList(), @@ -280,7 +281,7 @@ public RawUnionValue apply(@Nullable Object o) { }); } - private static class MultiOutputDoFn extends OldDoFn { + private static class MultiOutputDoFn extends DoFn { private TupleTag sideOutput1; private TupleTag sideOutput2; @@ -289,7 +290,7 @@ public MultiOutputDoFn(TupleTag sideOutput1, TupleTag sideOutput this.sideOutput2 = sideOutput2; } - @Override + @ProcessElement public void processElement(ProcessContext c) throws Exception { if (c.element().equals("one")) { c.sideOutput(sideOutput1, "side: one"); @@ -303,9 +304,9 @@ public void processElement(ProcessContext c) throws Exception { } } - private static class IdentityDoFn extends OldDoFn { - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception { + private static class IdentityDoFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { c.output(c.element()); } } 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 c6381ee4002e..663b910e3e80 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 @@ -24,8 +24,8 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -64,10 +64,8 @@ protected void postSubmit() throws Exception { /** * DoFn extracting user and timestamp. */ - public static class ExtractUserAndTimestamp extends OldDoFn, String> { - private static final long serialVersionUID = 0; - - @Override + private static class ExtractUserAndTimestamp extends DoFn, String> { + @ProcessElement public void processElement(ProcessContext c) { KV record = c.element(); int timestamp = record.getKey(); @@ -100,16 +98,16 @@ protected void testProgram() throws Exception { .withAllowedLateness(Duration.ZERO) .discardingFiredPanes()) - .apply(ParDo.of(new OldDoFn>() { - @Override + .apply(ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) throws Exception { String elem = c.element(); - c.output(KV.of((Void) null, elem)); + c.output(KV.of(null, elem)); } })) .apply(GroupByKey.create()) - .apply(ParDo.of(new OldDoFn>, String>() { - @Override + .apply(ParDo.of(new DoFn>, String>() { + @ProcessElement public void processElement(ProcessContext c) throws Exception { KV> elem = c.element(); StringBuilder str = new StringBuilder(); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java index 9410481f054e..9e6bba8fe0cb 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; @@ -100,8 +100,8 @@ protected void testProgram() throws Exception { - .apply(ParDo.of(new OldDoFn() { - @Override + .apply(ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) throws Exception { TableRow row = c.element(); long timestamp = (Integer) row.get("timestamp"); @@ -117,8 +117,8 @@ public void processElement(ProcessContext c) throws Exception { .apply(Count.perElement()); - PCollection format = output.apply(ParDo.of(new OldDoFn, String>() { - @Override + PCollection format = output.apply(ParDo.of(new DoFn, String>() { + @ProcessElement public void processElement(ProcessContext c) throws Exception { KV el = c.element(); String out = "user: " + el.getKey() + " value:" + el.getValue(); From af616d9741b19d0a7705df6fe075be1509aa659f Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 16:31:42 -0800 Subject: [PATCH 196/279] Remove ParDo.of(OldDoFn) from Apex runner The only such usage was of AssignWindowsDoFn. Now, instead, it is instantiated directly using a new translator for Window.Bound. This change also separates the overloads of ApexParDoOperator for old and new DoFn, to make the OldDoFn overload easier to track and later remove. --- .../apache/beam/runners/apex/ApexRunner.java | 46 +---------- .../translation/ApexPipelineTranslator.java | 2 + .../ParDoBoundMultiTranslator.java | 4 +- .../translation/ParDoBoundTranslator.java | 4 +- .../translation/WindowBoundTranslator.java | 78 +++++++++++++++++++ .../operators/ApexParDoOperator.java | 25 +++++- .../translation/ParDoBoundTranslatorTest.java | 3 +- .../beam/runners/core/AssignWindows.java | 46 ----------- .../translation/SparkAssignWindowFn.java | 2 +- 9 files changed, 108 insertions(+), 102 deletions(-) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java delete mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java index e5bde46274d5..f12ebef7b6e7 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -32,7 +32,6 @@ import org.apache.apex.api.Launcher.AppHandle; import org.apache.apex.api.Launcher.LaunchMode; import org.apache.beam.runners.apex.translation.ApexPipelineTranslator; -import org.apache.beam.runners.core.AssignWindows; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -46,9 +45,6 @@ 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.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; @@ -90,10 +86,7 @@ public static ApexRunner fromOptions(PipelineOptions options) { public OutputT apply( PTransform transform, InputT input) { - if (Window.Bound.class.equals(transform.getClass())) { - return (OutputT) ((PCollection) input).apply( - new AssignWindowsAndSetStrategy((Window.Bound) transform)); - } else if (Create.Values.class.equals(transform.getClass())) { + if (Create.Values.class.equals(transform.getClass())) { return (OutputT) PCollection .createPrimitiveOutputInternal( input.getPipeline(), @@ -162,43 +155,6 @@ public void populateDAG(DAG dag, Configuration conf) { } - /** - * copied from DirectPipelineRunner. - * used to replace Window.Bound till equivalent function is added in Apex - */ - private static class AssignWindowsAndSetStrategy - extends PTransform, PCollection> { - - private final Window.Bound wrapped; - - public AssignWindowsAndSetStrategy(Window.Bound wrapped) { - this.wrapped = wrapped; - } - - @Override - public PCollection expand(PCollection input) { - WindowingStrategy outputStrategy = - wrapped.getOutputStrategyInternal(input.getWindowingStrategy()); - - WindowFn windowFn = - (WindowFn) outputStrategy.getWindowFn(); - - // If the Window.Bound transform only changed parts other than the WindowFn, then - // we skip AssignWindows even though it should be harmless in a perfect world. - // The world is not perfect, and a GBK may have set it to InvalidWindows to forcibly - // crash if another GBK is performed without explicitly setting the WindowFn. So we skip - // AssignWindows in this case. - if (wrapped.getWindowFn() == null) { - return input.apply("Identity", ParDo.of(new IdentityFn())) - .setWindowingStrategyInternal(outputStrategy); - } else { - return input - .apply("AssignWindows", new AssignWindows<>(windowFn)) - .setWindowingStrategyInternal(outputStrategy); - } - } - } - private static class IdentityFn extends DoFn { private static final long serialVersionUID = 1L; @ProcessElement diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index 8d6db84d021f..c8e02908274c 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.slf4j.Logger; @@ -70,6 +71,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { new CreateApexPCollectionViewTranslator()); registerTransformTranslator(CreatePCollectionView.class, new CreatePCollectionViewTranslator()); + registerTransformTranslator(Window.Bound.class, new WindowBoundTranslator()); } public ApexPipelineTranslator(ApexPipelineOptions options) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java index 706482a5d07f..574ce8f067e8 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java @@ -31,7 +31,6 @@ import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; @@ -77,7 +76,6 @@ public void translate(ParDo.BoundMulti transform, TranslationCo ApexRunner.class.getSimpleName())); } - OldDoFn oldDoFn = transform.getFn(); PCollectionTuple output = context.getOutput(); PCollection input = context.getInput(); List> sideInputs = transform.getSideInputs(); @@ -89,7 +87,7 @@ public void translate(ParDo.BoundMulti transform, TranslationCo ApexParDoOperator operator = new ApexParDoOperator<>( context.getPipelineOptions(), - oldDoFn, + doFn, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), context.>getInput().getWindowingStrategy(), diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java index b5a50f671fd2..de786289c9aa 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java @@ -23,7 +23,6 @@ import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; @@ -64,7 +63,6 @@ public void translate(ParDo.Bound transform, TranslationContext ApexRunner.class.getSimpleName())); } - OldDoFn oldDoFn = transform.getOldFn(); PCollection output = context.getOutput(); PCollection input = context.getInput(); List> sideInputs = transform.getSideInputs(); @@ -76,7 +74,7 @@ public void translate(ParDo.Bound transform, TranslationContext ApexParDoOperator operator = new ApexParDoOperator<>( context.getPipelineOptions(), - oldDoFn, + doFn, new TupleTag(), TupleTagList.empty().getAll() /*sideOutputTags*/, output.getWindowingStrategy(), diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java new file mode 100644 index 000000000000..33b9269f9c43 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java @@ -0,0 +1,78 @@ +/* + * 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.apex.translation; + +import java.util.Collections; +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; +import org.apache.beam.runners.core.AssignWindowsDoFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +/** + * {@link Window.Bound} is translated to {link ApexParDoOperator} that wraps an {@link + * AssignWindowsDoFn}. + */ +class WindowBoundTranslator implements TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(Window.Bound transform, TranslationContext context) { + PCollection output = context.getOutput(); + PCollection input = context.getInput(); + @SuppressWarnings("unchecked") + WindowingStrategy windowingStrategy = + (WindowingStrategy) output.getWindowingStrategy(); + + OldDoFn fn = + (transform.getWindowFn() == null) + ? DoFnAdapters.toOldDoFn(new IdentityFn()) + : new AssignWindowsDoFn<>(transform.getWindowFn()); + + ApexParDoOperator operator = + new ApexParDoOperator( + context.getPipelineOptions().as(ApexPipelineOptions.class), + fn, + new TupleTag(), + TupleTagList.empty().getAll(), + windowingStrategy, + Collections.>emptyList(), + WindowedValue.getFullCoder( + input.getCoder(), windowingStrategy.getWindowFn().windowCoder()), + context.stateInternalsFactory()); + context.addOperator(operator, operator.output); + context.addStream(context.getInput(), operator.input); + } + + private static class IdentityFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.element()); + } + } +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java index 637c3ff29c07..08f062de512b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java @@ -28,11 +28,9 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; - import java.util.ArrayList; import java.util.List; import java.util.Map; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; @@ -50,6 +48,7 @@ import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; @@ -95,6 +94,7 @@ public class ApexParDoOperator extends BaseOperator implements private transient Map, DefaultOutputPort>> sideOutputPortMapping = Maps.newHashMapWithExpectedSize(5); + @Deprecated public ApexParDoOperator( ApexPipelineOptions pipelineOptions, OldDoFn doFn, @@ -125,6 +125,27 @@ public ApexParDoOperator( } + public ApexParDoOperator( + ApexPipelineOptions pipelineOptions, + DoFn doFn, + TupleTag mainOutputTag, + List> sideOutputTags, + WindowingStrategy windowingStrategy, + List> sideInputs, + Coder> inputCoder, + StateInternalsFactory stateInternalsFactory + ) { + this( + pipelineOptions, + DoFnAdapters.toOldDoFn(doFn), + mainOutputTag, + sideOutputTags, + windowingStrategy, + sideInputs, + inputCoder, + stateInternalsFactory); + } + @SuppressWarnings("unused") // for Kryo private ApexParDoOperator() { this.pipelineOptions = null; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java index 28b2ec95bb9c..fa94b2a442e9 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslatorTest.java @@ -49,7 +49,6 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.View; @@ -208,7 +207,7 @@ public void testSerialization() throws Exception { ApexParDoOperator operator = new ApexParDoOperator<>( options, - DoFnAdapters.toOldDoFn(new Add(singletonView)), + new Add(singletonView), new TupleTag(), TupleTagList.empty().getAll(), WindowingStrategy.globalDefault(), diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java deleted file mode 100644 index 375932aaf737..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java +++ /dev/null @@ -1,46 +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.core; - -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.values.PCollection; - -/** - * {@link PTransform} that uses privileged (non-user-facing) APIs to assign elements of a - * {@link PCollection} to windows according to the provided {@link WindowFn}. - * - * @param Type of elements being windowed - * @param Window type - */ -public class AssignWindows - extends PTransform, PCollection> { - - private WindowFn fn; - - public AssignWindows(WindowFn fn) { - this.fn = fn; - } - - @Override - public PCollection expand(PCollection input) { - return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn<>(fn))); - } -} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java index 9d7ed7d82915..18a3dd89990c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java @@ -29,7 +29,7 @@ /** - * An implementation of {@link org.apache.beam.runners.core.AssignWindows} for the Spark runner. + * An implementation of {@link org.apache.beam.runners.core.AssignWindowsDoFn} for the Spark runner. */ public class SparkAssignWindowFn implements Function, WindowedValue> { From 8330bfa74cd72e51a29649745e87a4f1a6e5ffa1 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 16:47:01 -0800 Subject: [PATCH 197/279] Pushes uses of OldDoFn deeper inside Flink runner In particular, various DoFnOperator's now take a regular DoFn rather than an OldDoFn, and convert it to an OldDoFn internally. This allows to remove uses of ParDo.getFn() returning OldDoFn. The only case where the OldDoFn inside a DoFnOperator is actually an OldDoFn rather than DoFn in disguise is now WindowDoFnOperator, which overrides getDoFn to return an actual GABW OldDoFn. --- .../FlinkBatchTransformTranslators.java | 9 +--- .../FlinkStreamingTransformTranslators.java | 8 ++-- .../functions/FlinkDoFnFunction.java | 10 +++-- .../FlinkMultiOutputDoFnFunction.java | 10 +++-- .../wrappers/streaming/DoFnOperator.java | 43 +++++++++++++++---- .../streaming/WindowDoFnOperator.java | 8 ++-- .../runners/flink/PipelineOptionsTest.java | 5 +-- .../flink/streaming/DoFnOperatorTest.java | 8 ++-- 8 files changed, 63 insertions(+), 38 deletions(-) diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index 9ac907f6eb9d..497b2931f796 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 @@ -50,7 +50,6 @@ 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.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -523,8 +522,6 @@ public void translateNode( DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - final OldDoFn oldDoFn = transform.getFn(); - TypeInformation> typeInformation = context.getTypeInfo(context.getOutput(transform)); @@ -539,7 +536,7 @@ public void translateNode( FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>( - oldDoFn, + doFn, context.getOutput(transform).getWindowingStrategy(), sideInputStrategies, context.getPipelineOptions()); @@ -570,8 +567,6 @@ public void translateNode( DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - final OldDoFn oldDoFn = transform.getFn(); - Map, PCollection> outputs = context.getOutput(transform).getAll(); Map, Integer> outputMap = Maps.newHashMap(); @@ -618,7 +613,7 @@ public void translateNode( @SuppressWarnings("unchecked") FlinkMultiOutputDoFnFunction doFnWrapper = new FlinkMultiOutputDoFnFunction( - oldDoFn, + doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(), 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 042f8dfc5a6f..42ef63098984 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 @@ -358,7 +358,7 @@ public void translateNode( if (sideInputs.isEmpty()) { DoFnOperator> doFnOperator = new DoFnOperator<>( - transform.getFn(), + transform.getNewFn(), inputTypeInfo, new TupleTag("main output"), Collections.>emptyList(), @@ -381,7 +381,7 @@ public void translateNode( DoFnOperator> doFnOperator = new DoFnOperator<>( - transform.getFn(), + transform.getNewFn(), inputTypeInfo, new TupleTag("main output"), Collections.>emptyList(), @@ -515,7 +515,7 @@ public void translateNode( if (sideInputs.isEmpty()) { DoFnOperator doFnOperator = new DoFnOperator<>( - transform.getFn(), + transform.getNewFn(), inputTypeInfo, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), @@ -542,7 +542,7 @@ public void translateNode( DoFnOperator doFnOperator = new DoFnOperator<>( - transform.getFn(), + transform.getNewFn(), inputTypeInfo, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), 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 db045f50a73e..ed200d58aa57 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 @@ -20,7 +20,10 @@ import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; @@ -46,16 +49,17 @@ public class FlinkDoFnFunction private final WindowingStrategy windowingStrategy; public FlinkDoFnFunction( - OldDoFn doFn, + DoFn doFn, WindowingStrategy windowingStrategy, Map, WindowingStrategy> sideInputs, PipelineOptions options) { - this.doFn = doFn; + this.doFn = DoFnAdapters.toOldDoFn(doFn); this.sideInputs = sideInputs; this.serializedOptions = new SerializedPipelineOptions(options); this.windowingStrategy = windowingStrategy; - this.requiresWindowAccess = doFn instanceof OldDoFn.RequiresWindowAccess; + this.requiresWindowAccess = + DoFnSignatures.signatureForDoFn(doFn).processElement().observesWindow(); this.hasSideInputs = !sideInputs.isEmpty(); } 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 7be4bb48ffa4..7f6a4369ad85 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 @@ -20,8 +20,11 @@ import java.util.Map; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; @@ -54,16 +57,17 @@ public class FlinkMultiOutputDoFnFunction private final WindowingStrategy windowingStrategy; public FlinkMultiOutputDoFnFunction( - OldDoFn doFn, + DoFn doFn, WindowingStrategy windowingStrategy, Map, WindowingStrategy> sideInputs, PipelineOptions options, Map, Integer> outputMap) { - this.doFn = doFn; + this.doFn = DoFnAdapters.toOldDoFn(doFn); this.serializedOptions = new SerializedPipelineOptions(options); this.outputMap = outputMap; - this.requiresWindowAccess = doFn instanceof OldDoFn.RequiresWindowAccess; + this.requiresWindowAccess = + DoFnSignatures.signatureForDoFn(doFn).processElement().observesWindow(); this.hasSideInputs = !sideInputs.isEmpty(); this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index a29664b5d3bc..6729aaa2ae38 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -40,6 +40,7 @@ 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.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -88,7 +89,8 @@ public class DoFnOperator implements OneInputStreamOperator, OutputT>, TwoInputStreamOperator, RawUnionValue, OutputT> { - protected OldDoFn doFn; + protected OldDoFn oldDoFn; + protected final SerializedPipelineOptions serializedOptions; protected final TupleTag mainOutputTag; @@ -117,8 +119,9 @@ public class DoFnOperator private transient Map> restoredSideInputState; + @Deprecated public DoFnOperator( - OldDoFn doFn, + OldDoFn oldDoFn, TypeInformation> inputType, TupleTag mainOutputTag, List> sideOutputTags, @@ -127,7 +130,7 @@ public DoFnOperator( Map> sideInputTagMapping, Collection> sideInputs, PipelineOptions options) { - this.doFn = doFn; + this.oldDoFn = oldDoFn; this.mainOutputTag = mainOutputTag; this.sideOutputTags = sideOutputTags; this.sideInputTagMapping = sideInputTagMapping; @@ -148,21 +151,43 @@ public DoFnOperator( setChainingStrategy(ChainingStrategy.ALWAYS); } + public DoFnOperator( + DoFn doFn, + TypeInformation> inputType, + TupleTag mainOutputTag, + List> sideOutputTags, + OutputManagerFactory outputManagerFactory, + WindowingStrategy windowingStrategy, + Map> sideInputTagMapping, + Collection> sideInputs, + PipelineOptions options) { + this( + DoFnAdapters.toOldDoFn(doFn), + inputType, + mainOutputTag, + sideOutputTags, + outputManagerFactory, + windowingStrategy, + sideInputTagMapping, + sideInputs, + options); + } + protected ExecutionContext.StepContext createStepContext() { return new StepContext(); } // allow overriding this in WindowDoFnOperator because this one dynamically creates // the DoFn - protected OldDoFn getDoFn() { - return doFn; + protected OldDoFn getOldDoFn() { + return oldDoFn; } @Override public void open() throws Exception { super.open(); - this.doFn = getDoFn(); + this.oldDoFn = getOldDoFn(); currentInputWatermark = Long.MIN_VALUE; currentOutputWatermark = currentInputWatermark; @@ -220,7 +245,7 @@ public Aggregator createAggregatorFor DoFnRunner doFnRunner = DoFnRunners.createDefault( serializedOptions.getPipelineOptions(), - doFn, + oldDoFn, sideInputReader, outputManagerFactory.create(output), mainOutputTag, @@ -232,13 +257,13 @@ public Aggregator createAggregatorFor pushbackDoFnRunner = PushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler); - doFn.setup(); + oldDoFn.setup(); } @Override public void close() throws Exception { super.close(); - doFn.teardown(); + oldDoFn.teardown(); } protected final long getPushbackWatermarkHold() { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index f2d7f1cc80c7..9cea5296d12b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -106,7 +106,7 @@ public WindowDoFnOperator( PipelineOptions options, Coder keyCoder) { super( - null, + (OldDoFn, KV>) null, inputType, mainOutputTag, sideOutputTags, @@ -124,7 +124,7 @@ public WindowDoFnOperator( } @Override - protected OldDoFn, KV> getDoFn() { + protected OldDoFn, KV> getOldDoFn() { StateInternalsFactory stateInternalsFactory = new StateInternalsFactory() { @Override public StateInternals stateInternalsForKey(K key) { @@ -138,10 +138,10 @@ public StateInternals stateInternalsForKey(K key) { // has the window type as generic parameter while WindowingStrategy is almost always // untyped. @SuppressWarnings("unchecked") - OldDoFn, KV> doFn = + OldDoFn, KV> oldDoFn = GroupAlsoByWindowViaWindowSetDoFn.create( windowingStrategy, stateInternalsFactory, (SystemReduceFn) systemReduceFn); - return doFn; + return oldDoFn; } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index e44a705ba750..4c97cc78230d 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; @@ -99,7 +98,7 @@ public void testNonNull() { @Test(expected = Exception.class) public void parDoBaseClassPipelineOptionsNullTest() { DoFnOperator doFnOperator = new DoFnOperator<>( - DoFnAdapters.toOldDoFn(new TestDoFn()), + new TestDoFn(), TypeInformation.of(new TypeHint>() {}), new TupleTag<>("main-output"), Collections.>emptyList(), @@ -118,7 +117,7 @@ public void parDoBaseClassPipelineOptionsNullTest() { public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { DoFnOperator doFnOperator = new DoFnOperator<>( - DoFnAdapters.toOldDoFn(new TestDoFn()), + new TestDoFn(), TypeInformation.of(new TypeHint>() {}), new TupleTag<>("main-output"), Collections.>emptyList(), diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index 65e244a9e7c6..113802ddabd3 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -35,8 +35,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PCollectionViewTesting; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -97,7 +95,7 @@ public void testSingleOutput() throws Exception { TupleTag outputTag = new TupleTag<>("main-output"); DoFnOperator doFnOperator = new DoFnOperator<>( - DoFnAdapters.toOldDoFn(new IdentityDoFn()), + new IdentityDoFn(), coderTypeInfo, outputTag, Collections.>emptyList(), @@ -141,7 +139,7 @@ public void testMultiOutputOutput() throws Exception { .build(); DoFnOperator doFnOperator = new DoFnOperator<>( - DoFnAdapters.toOldDoFn(new MultiOutputDoFn(sideOutput1, sideOutput2)), + new MultiOutputDoFn(sideOutput1, sideOutput2), coderTypeInfo, mainOutput, ImmutableList.>of(sideOutput1, sideOutput2), @@ -201,7 +199,7 @@ public void testSideInputs() throws Exception { .build(); DoFnOperator doFnOperator = new DoFnOperator<>( - DoFnAdapters.toOldDoFn(new IdentityDoFn()), + new IdentityDoFn(), coderTypeInfo, outputTag, Collections.>emptyList(), From e9e53c5d037561aa4dcacfcde69d76a03f3a1571 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 17:13:43 -0800 Subject: [PATCH 198/279] Removes OldDoFn from ParDo --- .../org/apache/beam/sdk/transforms/ParDo.java | 167 +++--------------- .../beam/sdk/transforms/OldDoFnTest.java | 125 ++++--------- 2 files changed, 55 insertions(+), 237 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 167f5faad7e7..d2149c054ef9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.HasDisplayData; @@ -530,23 +529,6 @@ public static Bound of(DoFn return new Unbound().of(fn, displayDataForFn(fn)); } - /** - * Creates a {@link ParDo} {@link PTransform} that will invoke the - * given {@link OldDoFn} function. - * - *

      The resulting {@link PTransform PTransform's} types have been bound, with the - * input being a {@code PCollection} and the output a - * {@code PCollection}, inferred from the types of the argument - * {@code OldDoFn}. It is ready to be applied, or further - * properties can be set on it first. - * - * @deprecated please port your {@link OldDoFn} to a {@link DoFn} - */ - @Deprecated - public static Bound of(OldDoFn fn) { - return new Unbound().of(fn, displayDataForFn(fn)); - } - private static DisplayData.ItemSpec> displayDataForFn(T fn) { return DisplayData.item("fn", fn.getClass()).withLabel("Transform Function"); } @@ -557,12 +539,7 @@ private static DisplayData.ItemSpec> displayDataForFn(T f * the {@link PCollection}. */ private static void validateWindowType( - PCollection input, Serializable fn) { - // No validation for OldDoFn - if (!(fn instanceof DoFn)) { - return; - } - + PCollection input, DoFn fn) { DoFnSignature signature = DoFnSignatures.getSignature((Class) fn.getClass()); TypeDescriptor actualWindowT = @@ -609,10 +586,6 @@ private static void validate(DoFn fn) { } } - private static OldDoFn adapt(DoFn fn) { - return DoFnAdapters.toOldDoFn(fn); - } - /** * An incomplete {@link ParDo} transform, with unbound input/output types. * @@ -688,24 +661,9 @@ public UnboundMulti withOutputTags( return new UnboundMulti<>(name, sideInputs, mainOutputTag, sideOutputTags); } - /** - * Returns a new {@link ParDo} {@link PTransform} that's like this - * transform but that will invoke the given {@link OldDoFn} - * function, and that has its input and output types bound. Does - * not modify this transform. The resulting {@link PTransform} is - * sufficiently specified to be applied, but more properties can - * still be specified. - * - * @deprecated please port your {@link OldDoFn} to a {@link DoFn} - */ - @Deprecated - public Bound of(OldDoFn oldFn) { - return of(oldFn, displayDataForFn(oldFn)); - } - private Bound of( - Serializable originalFn, DisplayData.ItemSpec> fnDisplayData) { - return new Bound<>(name, originalFn, sideInputs, fnDisplayData); + DoFn doFn, DisplayData.ItemSpec> fnDisplayData) { + return new Bound<>(name, doFn, sideInputs, fnDisplayData); } } @@ -725,12 +683,12 @@ public static class Bound extends PTransform, PCollection> { // Inherits name. private final List> sideInputs; - private final Serializable fn; + private final DoFn fn; private final DisplayData.ItemSpec> fnDisplayData; Bound( String name, - Serializable fn, + DoFn fn, List> sideInputs, DisplayData.ItemSpec> fnDisplayData) { super(name); @@ -787,7 +745,7 @@ public BoundMulti withOutputTags( @Override public PCollection expand(PCollection input) { checkArgument( - !isSplittable(getOldFn()), + !isSplittable(getNewFn()), "%s does not support Splittable DoFn", input.getPipeline().getOptions().getRunner().getName()); validateWindowType(input, fn); @@ -795,7 +753,7 @@ public PCollection expand(PCollection input) { input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setTypeDescriptor(getOldFn().getOutputTypeDescriptor()); + .setTypeDescriptor(getNewFn().getOutputTypeDescriptor()); } @Override @@ -803,14 +761,14 @@ public PCollection expand(PCollection input) { protected Coder getDefaultOutputCoder(PCollection input) throws CannotProvideCoderException { return input.getPipeline().getCoderRegistry().getDefaultCoder( - getOldFn().getOutputTypeDescriptor(), - getOldFn().getInputTypeDescriptor(), + getNewFn().getOutputTypeDescriptor(), + getNewFn().getInputTypeDescriptor(), ((PCollection) input).getCoder()); } @Override protected String getKindString() { - Class clazz = DoFnAdapters.getDoFnClass(getOldFn()); + Class clazz = getNewFn().getClass(); if (clazz.isAnonymousClass()) { return "AnonymousParDo"; } else { @@ -831,44 +789,7 @@ public void populateDisplayData(Builder builder) { ParDo.populateDisplayData(builder, (HasDisplayData) fn, fnDisplayData); } - /** - * @deprecated this method to be converted to return {@link DoFn}. If you want to receive - * an {@link OldDoFn} you should (temporarily) use {@link #getOldFn}. - */ - @Deprecated - public OldDoFn getFn() { - return getOldFn(); - } - - /** - * @deprecated please migrate to {@link #getNewFn} until {@link #getFn} is migrated to return - * a {@link DoFn}. - */ - @Deprecated - public OldDoFn getOldFn() { - if (fn instanceof OldDoFn) { - return (OldDoFn) fn; - } else { - return adapt((DoFn) fn); - } - } - public DoFn getNewFn() { - if (fn instanceof DoFn) { - return (DoFn) fn; - } else { - return ((OldDoFn) fn).toDoFn(); - } - } - - /** - * Returns the {@link OldDoFn} or {@link DoFn} used to create this transform. - * - * @deprecated for migration purposes only. There are some cases of {@link OldDoFn} that are not - * fully supported by wrapping it into a {@link DoFn}, such as {@link RequiresWindowAccess}. - */ - @Deprecated - public Object getOriginalFn() { return fn; } @@ -951,23 +872,8 @@ public BoundMulti of(DoFn fn) { return of(fn, displayDataForFn(fn)); } - /** - * Returns a new multi-output {@link ParDo} {@link PTransform} - * that's like this transform but that will invoke the given - * {@link OldDoFn} function, and that has its input type bound. - * Does not modify this transform. The resulting - * {@link PTransform} is sufficiently specified to be applied, but - * more properties can still be specified. - * - * @deprecated please port your {@link OldDoFn} to a {@link DoFn} - */ - @Deprecated - public BoundMulti of(OldDoFn fn) { - return of(fn, displayDataForFn(fn)); - } - private BoundMulti of( - Serializable fn, DisplayData.ItemSpec> fnDisplayData) { + DoFn fn, DisplayData.ItemSpec> fnDisplayData) { return new BoundMulti<>(name, fn, sideInputs, mainOutputTag, sideOutputTags, fnDisplayData); } } @@ -990,11 +896,11 @@ public static class BoundMulti private final TupleTag mainOutputTag; private final TupleTagList sideOutputTags; private final DisplayData.ItemSpec> fnDisplayData; - private final Serializable fn; + private final DoFn fn; BoundMulti( String name, - Serializable fn, + DoFn fn, List> sideInputs, TupleTag mainOutputTag, TupleTagList sideOutputTags, @@ -1046,7 +952,7 @@ public BoundMulti withSideInputs( @Override public PCollectionTuple expand(PCollection input) { checkArgument( - !isSplittable(getOldFn()), + !isSplittable(getNewFn()), "%s does not support Splittable DoFn", input.getPipeline().getOptions().getRunner().getName()); validateWindowType(input, fn); @@ -1059,7 +965,7 @@ public PCollectionTuple expand(PCollection input) { // The fn will likely be an instance of an anonymous subclass // such as DoFn { }, thus will have a high-fidelity // TypeDescriptor for the output type. - outputs.get(mainOutputTag).setTypeDescriptor(getOldFn().getOutputTypeDescriptor()); + outputs.get(mainOutputTag).setTypeDescriptor(getNewFn().getOutputTypeDescriptor()); return outputs; } @@ -1084,7 +990,7 @@ public Coder getDefaultOutputCoder( @Override protected String getKindString() { - Class clazz = DoFnAdapters.getDoFnClass(getOldFn()); + Class clazz = getNewFn().getClass(); if (clazz.isAnonymousClass()) { return "AnonymousParMultiDo"; } else { @@ -1095,37 +1001,11 @@ protected String getKindString() { @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - ParDo.populateDisplayData(builder, (HasDisplayData) fn, fnDisplayData); - } - - /** - * @deprecated this method to be converted to return {@link DoFn}. If you want to receive - * an {@link OldDoFn} you should (temporarily) use {@link #getOldFn}. - */ - @Deprecated - public OldDoFn getFn() { - return getOldFn(); - } - - /** - * @deprecated please migrate to {@link #getNewFn} until {@link #getFn} is migrated to return - * a {@link DoFn}. - */ - @Deprecated - public OldDoFn getOldFn() { - if (fn instanceof OldDoFn) { - return (OldDoFn) fn; - } else { - return adapt((DoFn) fn); - } + ParDo.populateDisplayData(builder, fn, fnDisplayData); } public DoFn getNewFn() { - if (fn instanceof DoFn) { - return (DoFn) fn; - } else { - return ((OldDoFn) fn).toDoFn(); - } + return fn; } public TupleTag getMainOutputTag() { @@ -1148,14 +1028,7 @@ private static void populateDisplayData( builder.include("fn", fn).add(fnDisplayData); } - private static boolean isSplittable(OldDoFn oldDoFn) { - DoFn fn = DoFnAdapters.getDoFn(oldDoFn); - if (fn == null) { - return false; - } - return DoFnSignatures - .getSignature(fn.getClass()) - .processElement() - .isSplittable(); + private static boolean isSplittable(DoFn fn) { + return DoFnSignatures.signatureForDoFn(fn).processElement().isSplittable(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java index 07e3078e3f1c..cc842528f927 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/OldDoFnTest.java @@ -18,28 +18,20 @@ package org.apache.beam.sdk.transforms; import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; import java.io.Serializable; -import java.util.Map; -import org.apache.beam.sdk.AggregatorValues; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.Pipeline.PipelineExecutionException; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.testing.NeedsRunner; -import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Max.MaxIntegerFn; -import org.apache.beam.sdk.transforms.Sum.SumIntegerFn; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -134,68 +126,52 @@ public void testCreateAggregatorsWithDifferentNamesSucceeds() { } @Test - @Category(NeedsRunner.class) - public void testCreateAggregatorInStartBundleThrows() { - TestPipeline p = createTestPipeline(new OldDoFn() { + public void testCreateAggregatorThrowsWhenAggregatorsAreFinal() throws Exception { + OldDoFn fn = new OldDoFn() { @Override - public void startBundle(OldDoFn.Context c) throws Exception { - createAggregator("anyAggregate", new MaxIntegerFn()); - } - - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception {} - }); - - thrown.expect(PipelineExecutionException.class); - thrown.expectCause(isA(IllegalStateException.class)); + public void processElement(ProcessContext c) throws Exception { } + }; + OldDoFn.Context context = createContext(fn); + context.setupDelegateAggregators(); - p.run(); + thrown.expect(isA(IllegalStateException.class)); + fn.createAggregator("anyAggregate", new MaxIntegerFn()); } - @Test - @Category(NeedsRunner.class) - public void testCreateAggregatorInProcessElementThrows() { - TestPipeline p = createTestPipeline(new OldDoFn() { + private OldDoFn.Context createContext(OldDoFn fn) { + return fn.new Context() { @Override - public void processElement(ProcessContext c) throws Exception { - createAggregator("anyAggregate", new MaxIntegerFn()); + public PipelineOptions getPipelineOptions() { + throw new UnsupportedOperationException(); } - }); - - thrown.expect(PipelineExecutionException.class); - thrown.expectCause(isA(IllegalStateException.class)); - - p.run(); - } - @Test - @Category(NeedsRunner.class) - public void testCreateAggregatorInFinishBundleThrows() { - TestPipeline p = createTestPipeline(new OldDoFn() { @Override - public void finishBundle(OldDoFn.Context c) throws Exception { - createAggregator("anyAggregate", new MaxIntegerFn()); + public void output(String output) { + throw new UnsupportedOperationException(); } @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception {} - }); - - thrown.expect(PipelineExecutionException.class); - thrown.expectCause(isA(IllegalStateException.class)); + public void outputWithTimestamp(String output, Instant timestamp) { + throw new UnsupportedOperationException(); + } - p.run(); - } + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException(); + } - /** - * Initialize a test pipeline with the specified {@link OldDoFn}. - */ - private TestPipeline createTestPipeline(OldDoFn fn) { - TestPipeline pipeline = TestPipeline.create(); - pipeline.apply(Create.of((InputT) null)) - .apply(ParDo.of(fn)); + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + throw new UnsupportedOperationException(); + } - return pipeline; + @Override + public + Aggregator createAggregatorInternal( + String name, CombineFn combiner) { + throw new UnsupportedOperationException(); + } + }; } @Test @@ -209,35 +185,4 @@ public void processElement(ProcessContext c) throws Exception {} DisplayData data = DisplayData.from(usesDefault); assertThat(data.items(), empty()); } - - @Test - @Category(NeedsRunner.class) - public void testAggregators() throws Exception { - Pipeline pipeline = TestPipeline.create(); - - CountOddsFn countOdds = new CountOddsFn(); - PCollection output = pipeline - .apply(Create.of(1, 3, 5, 7, 2, 4, 6, 8, 10, 12, 14, 20, 42, 68, 100)) - .apply(ParDo.of(countOdds)); - PipelineResult result = pipeline.run(); - - AggregatorValues values = result.getAggregatorValues(countOdds.aggregator); - - Map valuesMap = values.getValuesAtSteps(); - - assertThat(valuesMap.size(), equalTo(1)); - assertThat(valuesMap.get(output.getProducingTransformInternal().getFullName()), equalTo(4)); - } - - private static class CountOddsFn extends OldDoFn { - @Override - public void processElement(ProcessContext c) throws Exception { - if (c.element() % 2 == 1) { - aggregator.addValue(1); - } - } - - Aggregator aggregator = - createAggregator("odds", new SumIntegerFn()); - } } From a22de15012c51e8b7e31143021f0a298e093bf51 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 17:21:40 -0800 Subject: [PATCH 199/279] Removes code for wrapping DoFn as an OldDoFn --- .../beam/sdk/transforms/DoFnAdapters.java | 150 --------- .../apache/beam/sdk/transforms/OldDoFn.java | 295 +----------------- .../sdk/transforms/reflect/DoFnInvokers.java | 141 +-------- .../transforms/reflect/DoFnInvokersTest.java | 36 --- 4 files changed, 11 insertions(+), 611 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index e15b08b46d79..d1c40a69f54f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.transforms; import java.io.IOException; -import java.util.Collection; -import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn.Context; @@ -38,7 +36,6 @@ import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; @@ -53,18 +50,6 @@ public class DoFnAdapters { /** Should not be instantiated. */ private DoFnAdapters() {} - /** - * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the - * original {@link DoFn}, otherwise returns {@code fn.getClass()}. - */ - public static Class getDoFnClass(OldDoFn fn) { - if (fn instanceof SimpleDoFnAdapter) { - return ((SimpleDoFnAdapter) fn).fn.getClass(); - } else { - return fn.getClass(); - } - } - /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */ @SuppressWarnings({"unchecked", "rawtypes"}) public static OldDoFn toOldDoFn(DoFn fn) { @@ -76,126 +61,6 @@ public static OldDoFn toOldDoFn(DoFn OldDoFn.ProcessContext adaptProcessContext( - OldDoFn fn, - final DoFn.ProcessContext c, - final DoFnInvoker.ArgumentProvider extra) { - return fn.new ProcessContext() { - @Override - public InputT element() { - return c.element(); - } - - @Override - public T sideInput(PCollectionView view) { - return c.sideInput(view); - } - - @Override - public Instant timestamp() { - return c.timestamp(); - } - - @Override - public BoundedWindow window() { - return extra.window(); - } - - @Override - public PaneInfo pane() { - return c.pane(); - } - - @Override - public WindowingInternals windowingInternals() { - return extra.windowingInternals(); - } - - @Override - public PipelineOptions getPipelineOptions() { - return c.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - c.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - c.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - c.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - c.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - protected Aggregator createAggregatorInternal( - String name, CombineFn combiner) { - return c.createAggregator(name, combiner); - } - }; - } - - /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */ - public static OldDoFn.Context adaptContext( - OldDoFn fn, - final DoFn.Context c) { - return fn.new Context() { - @Override - public PipelineOptions getPipelineOptions() { - return c.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - c.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - c.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - c.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - c.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - protected Aggregator createAggregatorInternal( - String name, CombineFn combiner) { - return c.createAggregator(name, combiner); - } - }; - } - - /** - * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise, - * returns {@code null}. - */ - @Nullable - public static DoFn getDoFn(OldDoFn fn) { - if (fn instanceof SimpleDoFnAdapter) { - return ((SimpleDoFnAdapter) fn).fn; - } else { - return null; - } - } - /** * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link * OldDoFn}. @@ -237,21 +102,6 @@ public void processElement(ProcessContext c) throws Exception { invoker.invokeProcessElement(adapter); } - @Override - protected TypeDescriptor getInputTypeDescriptor() { - return fn.getInputTypeDescriptor(); - } - - @Override - protected TypeDescriptor getOutputTypeDescriptor() { - return fn.getOutputTypeDescriptor(); - } - - @Override - Collection> getAggregators() { - return fn.getAggregators(); - } - @Override public Duration getAllowedTimestampSkew() { return fn.getAllowedTimestampSkew(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index 2d2c1fde737b..0aef552f1c57 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; @@ -71,21 +70,6 @@ */ @Deprecated public abstract class OldDoFn implements Serializable, HasDisplayData { - - public DoFn toDoFn() { - DoFn doFn = DoFnAdapters.getDoFn(this); - if (doFn != null) { - return doFn; - } - if (this instanceof RequiresWindowAccess) { - // No parameters as it just accesses `this` - return new AdaptedRequiresWindowAccessDoFn(); - } else { - // No parameters as it just accesses `this` - return new AdaptedDoFn(); - } - } - /** * Information accessible to all methods in this {@code OldDoFn}. * Used primarily to output elements. @@ -334,7 +318,7 @@ public OldDoFn() { this(new HashMap>()); } - OldDoFn(Map> aggregators) { + public OldDoFn(Map> aggregators) { this.aggregators = aggregators; } @@ -418,32 +402,6 @@ public void populateDisplayData(DisplayData.Builder builder) { ///////////////////////////////////////////////////////////////////////////// - /** - * Returns a {@link TypeDescriptor} capturing what is known statically - * about the input type of this {@code OldDoFn} instance's most-derived - * class. - * - *

      See {@link #getOutputTypeDescriptor} for more discussion. - */ - protected TypeDescriptor getInputTypeDescriptor() { - return new TypeDescriptor(getClass()) {}; - } - - /** - * Returns a {@link TypeDescriptor} capturing what is known statically - * about the output type of this {@code OldDoFn} instance's - * most-derived class. - * - *

      In the normal case of a concrete {@code OldDoFn} subclass with - * no generic type parameters of its own (including anonymous inner - * classes), this will be a complete non-generic type, which is good - * for choosing a default output {@code Coder} for the output - * {@code PCollection}. - */ - protected TypeDescriptor getOutputTypeDescriptor() { - return new TypeDescriptor(getClass()) {}; - } - /** * Returns an {@link Aggregator} with aggregation logic specified by the * {@link CombineFn} argument. The name provided must be unique across @@ -504,255 +462,4 @@ protected final Aggregator createAggregator(St Collection> getAggregators() { return Collections.>unmodifiableCollection(aggregators.values()); } - - /** - * A {@link Context} for an {@link OldDoFn} via a context for a proper {@link DoFn}. - */ - private class AdaptedContext extends Context { - - private final DoFn.Context newContext; - - public AdaptedContext( - DoFn.Context newContext) { - this.newContext = newContext; - super.setupDelegateAggregators(); - } - - @Override - public PipelineOptions getPipelineOptions() { - return newContext.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - newContext.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - newContext.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - newContext.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - newContext.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - protected Aggregator createAggregatorInternal( - String name, CombineFn combiner) { - return newContext.createAggregator(name, combiner); - } - } - - /** - * A {@link ProcessContext} for an {@link OldDoFn} via a context for a proper {@link DoFn}. - */ - private class AdaptedProcessContext extends ProcessContext { - - private final DoFn.ProcessContext newContext; - - public AdaptedProcessContext(DoFn.ProcessContext newContext) { - this.newContext = newContext; - } - - @Override - public InputT element() { - return newContext.element(); - } - - @Override - public T sideInput(PCollectionView view) { - return newContext.sideInput(view); - } - - @Override - public Instant timestamp() { - return newContext.timestamp(); - } - - @Override - public BoundedWindow window() { - throw new UnsupportedOperationException(String.format( - "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s", - OldDoFn.class.getSimpleName(), - OldDoFn.ProcessContext.class.getSimpleName(), - OldDoFn.class.getSimpleName(), - DoFn.class.getSimpleName())); - } - - @Override - public PaneInfo pane() { - return newContext.pane(); - } - - @Override - public WindowingInternals windowingInternals() { - throw new UnsupportedOperationException(String.format( - "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s", - OldDoFn.class.getSimpleName(), - OldDoFn.ProcessContext.class.getSimpleName(), - OldDoFn.class.getSimpleName(), - DoFn.class.getSimpleName())); - } - - @Override - public PipelineOptions getPipelineOptions() { - return newContext.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - newContext.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - newContext.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - newContext.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - newContext.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - protected Aggregator createAggregatorInternal( - String name, CombineFn combiner) { - return newContext.createAggregator(name, combiner); - } - } - - private class AdaptedDoFn extends DoFn { - - @Setup - public void setup() throws Exception { - OldDoFn.this.setup(); - } - - @StartBundle - public void startBundle(Context c) throws Exception { - OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c)); - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - OldDoFn.this.processElement(OldDoFn.this.new AdaptedProcessContext(c)); - } - - @FinishBundle - public void finishBundle(Context c) throws Exception { - OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c)); - } - - @Teardown - public void teardown() throws Exception { - OldDoFn.this.teardown(); - } - - @Override - public Duration getAllowedTimestampSkew() { - return OldDoFn.this.getAllowedTimestampSkew(); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - OldDoFn.this.populateDisplayData(builder); - } - - @Override - public TypeDescriptor getInputTypeDescriptor() { - return OldDoFn.this.getInputTypeDescriptor(); - } - - @Override - Collection> getAggregators() { - return OldDoFn.this.getAggregators(); - } - - @Override - public TypeDescriptor getOutputTypeDescriptor() { - return OldDoFn.this.getOutputTypeDescriptor(); - } - } - - /** - * A {@link ProcessContext} for an {@link OldDoFn} that implements - * {@link RequiresWindowAccess}, via a context for a proper {@link DoFn}. - */ - private class AdaptedRequiresWindowAccessProcessContext extends AdaptedProcessContext { - - private final BoundedWindow window; - - public AdaptedRequiresWindowAccessProcessContext( - DoFn.ProcessContext newContext, - BoundedWindow window) { - super(newContext); - this.window = window; - } - - @Override - public BoundedWindow window() { - return window; - } - } - - private class AdaptedRequiresWindowAccessDoFn extends DoFn { - - @Setup - public void setup() throws Exception { - OldDoFn.this.setup(); - } - - @StartBundle - public void startBundle(Context c) throws Exception { - OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c)); - } - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { - OldDoFn.this.processElement( - OldDoFn.this.new AdaptedRequiresWindowAccessProcessContext(c, window)); - } - - @FinishBundle - public void finishBundle(Context c) throws Exception { - OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c)); - } - - @Teardown - public void teardown() throws Exception { - OldDoFn.this.teardown(); - } - - @Override - public Duration getAllowedTimestampSkew() { - return OldDoFn.this.getAllowedTimestampSkew(); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - OldDoFn.this.populateDisplayData(builder); - } - - @Override - public TypeDescriptor getInputTypeDescriptor() { - return OldDoFn.this.getInputTypeDescriptor(); - } - - @Override - public TypeDescriptor getOutputTypeDescriptor() { - return OldDoFn.this.getOutputTypeDescriptor(); - } - } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index 50a7082cdad5..b141d51150ba 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -18,13 +18,7 @@ package org.apache.beam.sdk.transforms.reflect; import java.io.Serializable; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.util.UserCodeException; /** Static utilities for working with {@link DoFnInvoker}. */ public class DoFnInvokers { @@ -42,137 +36,22 @@ public static DoFnInvoker invokerFor( return ByteBuddyDoFnInvokerFactory.only().newByteBuddyInvoker(fn); } - private DoFnInvokers() {} - /** - * Returns a {@link DoFnInvoker} for the given {@link Object}, which should be either a {@link - * DoFn} or an {@link OldDoFn}. The expected use would be to deserialize a user's function as an - * {@link Object} and then pass it to this method, so there is no need to statically specify what - * sort of object it is. + * Temporarily retained for compatibility with Dataflow worker. + * TODO: delete this when Dataflow worker is fixed to call {@link #invokerFor(DoFn)}. * - * @deprecated this is to be used only as a migration path for decoupling upgrades + * @deprecated Use {@link #invokerFor(DoFn)}. */ + @SuppressWarnings("unchecked") @Deprecated - public static DoFnInvoker invokerFor(Serializable deserializedFn) { + public static DoFnInvoker invokerFor( + Serializable deserializedFn) { if (deserializedFn instanceof DoFn) { - return invokerFor((DoFn) deserializedFn); - } else if (deserializedFn instanceof OldDoFn) { - return new OldDoFnInvoker<>((OldDoFn) deserializedFn); - } else { - throw new IllegalArgumentException( - String.format( - "Cannot create a %s for %s; it should be either a %s or an %s.", - DoFnInvoker.class.getSimpleName(), - deserializedFn.toString(), - DoFn.class.getSimpleName(), - OldDoFn.class.getSimpleName())); + return invokerFor((DoFn) deserializedFn); } + throw new UnsupportedOperationException( + "Only DoFn supported, was: " + deserializedFn.getClass()); } - /** @deprecated use {@link DoFnInvokers#invokerFor(DoFn)}. */ - @Deprecated public static final DoFnInvokers INSTANCE = new DoFnInvokers(); - - /** @deprecated use {@link DoFnInvokers#invokerFor(DoFn)}. */ - @Deprecated - public DoFnInvoker invokerFor(Object deserializedFn) { - return (DoFnInvoker) DoFnInvokers.invokerFor((Serializable) deserializedFn); - } - - - static class OldDoFnInvoker implements DoFnInvoker { - - private final OldDoFn fn; - - public OldDoFnInvoker(OldDoFn fn) { - this.fn = fn; - } - - @Override - public DoFn.ProcessContinuation invokeProcessElement( - ArgumentProvider extra) { - // The outer DoFn is immaterial - it exists only to avoid typing InputT and OutputT repeatedly - DoFn.ProcessContext newCtx = - extra.processContext(new DoFn() {}); - OldDoFn.ProcessContext oldCtx = - DoFnAdapters.adaptProcessContext(fn, newCtx, extra); - try { - fn.processElement(oldCtx); - return DoFn.ProcessContinuation.stop(); - } catch (Throwable exc) { - throw UserCodeException.wrap(exc); - } - } - - @Override - public void invokeOnTimer(String timerId, ArgumentProvider arguments) { - throw new UnsupportedOperationException( - String.format("Timers are not supported for %s", OldDoFn.class.getSimpleName())); - } - - @Override - public void invokeStartBundle(DoFn.Context c) { - OldDoFn.Context oldCtx = DoFnAdapters.adaptContext(fn, c); - try { - fn.startBundle(oldCtx); - } catch (Throwable exc) { - throw UserCodeException.wrap(exc); - } - } - - @Override - public void invokeFinishBundle(DoFn.Context c) { - OldDoFn.Context oldCtx = DoFnAdapters.adaptContext(fn, c); - try { - fn.finishBundle(oldCtx); - } catch (Throwable exc) { - throw UserCodeException.wrap(exc); - } - } - - @Override - public void invokeSetup() { - try { - fn.setup(); - } catch (Throwable exc) { - throw UserCodeException.wrap(exc); - } - } - - @Override - public void invokeTeardown() { - try { - fn.teardown(); - } catch (Throwable exc) { - throw UserCodeException.wrap(exc); - } - } - - @Override - public RestrictionT invokeGetInitialRestriction(InputT element) { - throw new UnsupportedOperationException("OldDoFn is not splittable"); - } - - @Override - public Coder invokeGetRestrictionCoder( - CoderRegistry coderRegistry) { - throw new UnsupportedOperationException("OldDoFn is not splittable"); - } - - @Override - public void invokeSplitRestriction( - InputT element, RestrictionT restriction, DoFn.OutputReceiver receiver) { - throw new UnsupportedOperationException("OldDoFn is not splittable"); - } - - @Override - public > - TrackerT invokeNewTracker(RestrictionT restriction) { - throw new UnsupportedOperationException("OldDoFn is not splittable"); - } - - @Override - public DoFn getFn() { - throw new UnsupportedOperationException("getFn is not supported for OldDoFn"); - } - } + private DoFnInvokers() {} } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 456a6ebff0a6..55b8e7efb244 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -25,7 +25,6 @@ import static org.junit.Assert.assertThat; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; -import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -738,39 +737,4 @@ public void onMyTimer(IntervalWindow w) { invoker.invokeOnTimer(timerId, mockArgumentProvider); assertThat(fn.window, equalTo(testWindow)); } - - private class OldDoFnIdentity extends OldDoFn { - public void processElement(ProcessContext c) {} - } - - @Test - public void testOldDoFnProcessElement() throws Exception { - new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn) - .invokeProcessElement(mockArgumentProvider); - verify(mockOldDoFn).processElement(any(OldDoFn.ProcessContext.class)); - } - - @Test - public void testOldDoFnStartBundle() throws Exception { - new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeStartBundle(mockProcessContext); - verify(mockOldDoFn).startBundle(any(OldDoFn.Context.class)); - } - - @Test - public void testOldDoFnFinishBundle() throws Exception { - new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeFinishBundle(mockProcessContext); - verify(mockOldDoFn).finishBundle(any(OldDoFn.Context.class)); - } - - @Test - public void testOldDoFnSetup() throws Exception { - new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeSetup(); - verify(mockOldDoFn).setup(); - } - - @Test - public void testOldDoFnTeardown() throws Exception { - new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeTeardown(); - verify(mockOldDoFn).teardown(); - } } From f3e8a0383bf9cb3f9452e0364f7deba113cadff9 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 17:23:15 -0800 Subject: [PATCH 200/279] Removes ArgumentProvider.windowingInternals --- .../beam/runners/core/SimpleDoFnRunner.java | 57 ------------------- .../beam/runners/core/SplittableParDo.java | 7 --- .../beam/sdk/transforms/DoFnAdapters.java | 14 ----- .../beam/sdk/transforms/DoFnTester.java | 7 --- .../sdk/transforms/reflect/DoFnInvoker.java | 20 ------- .../transforms/reflect/DoFnInvokersTest.java | 6 -- 6 files changed, 111 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index af7f5cae571e..041cdde69292 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -52,13 +52,10 @@ import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; 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.State; -import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateSpec; @@ -419,11 +416,6 @@ public OutputReceiver outputReceiver() { throw new UnsupportedOperationException("OutputReceiver is for testing only."); } - @Override - public WindowingInternals windowingInternals() { - throw new UnsupportedOperationException("WindowingInternals are unsupported."); - } - @Override public RestrictionTracker restrictionTracker() { throw new UnsupportedOperationException( @@ -634,54 +626,5 @@ public Timer timer(String timerId) { throw new UnsupportedOperationException("Timer parameters are not supported."); } - @Override - public WindowingInternals windowingInternals() { - return new WindowingInternals() { - @Override - public Collection windows() { - return windowedValue.getWindows(); - } - - @Override - public PaneInfo pane() { - return windowedValue.getPane(); - } - - @Override - public TimerInternals timerInternals() { - return context.stepContext.timerInternals(); - } - - @Override - public StateInternals stateInternals() { - return stepContext.stateInternals(); - } - - @Override - public void outputWindowedValue( - OutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - throw new UnsupportedOperationException("A DoFn cannot output to a different window"); - } - - @Override - public void sideOutputWindowedValue( - TupleTag tag, - SideOutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - throw new UnsupportedOperationException( - "A DoFn cannot side output to a different window"); - } - - @Override - public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { - return context.sideInput(view, sideInputWindow); - } - }; - } } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 8a9bfcd205f5..720db63137f7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -53,7 +53,6 @@ import org.apache.beam.sdk.util.Timer; 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.State; import org.apache.beam.sdk.util.state.StateInternals; @@ -684,12 +683,6 @@ public DoFn.OutputReceiver outputReceiver() { throw new IllegalStateException("Unexpected extra context access on a splittable DoFn"); } - @Override - public WindowingInternals windowingInternals() { - // DoFnSignatures should have verified that this DoFn doesn't access extra context. - throw new IllegalStateException("Unexpected extra context access on a splittable DoFn"); - } - @Override public TrackerT restrictionTracker() { return tracker; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index d1c40a69f54f..0a71faab2815 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; @@ -201,14 +200,6 @@ public OnTimerContext onTimerContext(DoFn doFn) { "Timers are not supported for OldDoFn"); } - @Override - public WindowingInternals windowingInternals() { - // The OldDoFn doesn't allow us to ask for these outside ProcessElements, so this - // should be unreachable. - throw new UnsupportedOperationException( - "Can only get WindowingInternals in processElement"); - } - @Override public DoFn.InputProvider inputProvider() { throw new UnsupportedOperationException("inputProvider() exists only for testing"); @@ -321,11 +312,6 @@ public OnTimerContext onTimerContext(DoFn doFn) { throw new UnsupportedOperationException("Timers are not supported for OldDoFn"); } - @Override - public WindowingInternals windowingInternals() { - return context.windowingInternals(); - } - @Override public DoFn.InputProvider inputProvider() { throw new UnsupportedOperationException("inputProvider() exists only for testing"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 93b3f5954898..527d529cda4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -49,7 +49,6 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; @@ -334,12 +333,6 @@ public DoFn.OutputReceiver outputReceiver() { "Not expected to access OutputReceiver from DoFnTester"); } - @Override - public WindowingInternals windowingInternals() { - throw new UnsupportedOperationException( - "Not expected to access WindowingInternals from a new DoFn"); - } - @Override public RestrictionTracker restrictionTracker() { throw new UnsupportedOperationException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index 97ac9d3d53c7..354578e2a831 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -27,11 +27,9 @@ import org.apache.beam.sdk.transforms.DoFn.StartBundle; import org.apache.beam.sdk.transforms.DoFn.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.State; /** @@ -121,19 +119,6 @@ interface ArgumentProvider { /** A placeholder for testing purposes. */ OutputReceiver outputReceiver(); - /** - * For migration from {@link OldDoFn} to {@link DoFn}, provide a {@link WindowingInternals} so - * an {@link OldDoFn} can be run via {@link DoFnInvoker}. - * - *

      This is not exposed via the reflective capabilities of {@link DoFn}. - * - * @deprecated Please port occurences of {@link OldDoFn} to {@link DoFn}. If they require state - * and timers, they will need to wait for the arrival of those features. Do not introduce - * new uses of this method. - */ - @Deprecated - WindowingInternals windowingInternals(); - /** * If this is a splittable {@link DoFn}, returns the {@link RestrictionTracker} associated with * the current {@link ProcessElement} call. @@ -179,11 +164,6 @@ public OutputReceiver outputReceiver() { return null; } - @Override - public WindowingInternals windowingInternals() { - return null; - } - @Override public State state(String stateId) { return null; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 55b8e7efb244..4c6bee13f79d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -40,7 +40,6 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker.FakeArgumentProvider; import org.apache.beam.sdk.transforms.reflect.testhelper.DoFnInvokersTestHelper; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; @@ -51,7 +50,6 @@ import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; @@ -77,18 +75,14 @@ public class DoFnInvokersTest { @Mock private IntervalWindow mockWindow; @Mock private DoFn.InputProvider mockInputProvider; @Mock private DoFn.OutputReceiver mockOutputReceiver; - @Mock private WindowingInternals mockWindowingInternals; @Mock private DoFnInvoker.ArgumentProvider mockArgumentProvider; - @Mock private OldDoFn mockOldDoFn; - @Before public void setUp() { MockitoAnnotations.initMocks(this); when(mockArgumentProvider.window()).thenReturn(mockWindow); when(mockArgumentProvider.inputProvider()).thenReturn(mockInputProvider); when(mockArgumentProvider.outputReceiver()).thenReturn(mockOutputReceiver); - when(mockArgumentProvider.windowingInternals()).thenReturn(mockWindowingInternals); when(mockArgumentProvider.processContext(Matchers.any())).thenReturn(mockProcessContext); } From ca1dd7a3a4a01e2696a8131809c5798aef55d6a0 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 17:24:23 -0800 Subject: [PATCH 201/279] Removes unused code from NoOpOldDoFn --- .../beam/sdk/transforms/NoOpOldDoFn.java | 72 ------------------- 1 file changed, 72 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java index 302b66aa7c4e..504480b66cea 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java @@ -19,10 +19,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowingInternals; -import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Instant; @@ -45,13 +41,6 @@ public OldDoFn.Context context() { return new NoOpDoFnContext(); } - /** - * Returns a new NoOp Process Context. - */ - public OldDoFn.ProcessContext processContext() { - return new NoOpDoFnProcessContext(); - } - /** * A {@link OldDoFn.Context} that does nothing and returns exclusively null. */ @@ -79,65 +68,4 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, return null; } } - - /** - * A {@link OldDoFn.ProcessContext} that does nothing and returns exclusively - * null. - */ - private class NoOpDoFnProcessContext extends OldDoFn.ProcessContext { - @Override - public InputT element() { - return null; - } - - @Override - public T sideInput(PCollectionView view) { - return null; - } - - @Override - public Instant timestamp() { - return null; - } - - @Override - public BoundedWindow window() { - return null; - } - - @Override - public PaneInfo pane() { - return null; - } - - @Override - public WindowingInternals windowingInternals() { - return null; - } - - @Override - public PipelineOptions getPipelineOptions() { - return null; - } - - @Override - public void output(OutputT output) {} - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) {} - - @Override - public void sideOutput(TupleTag tag, T output) {} - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, - Instant timestamp) {} - - @Override - protected Aggregator - createAggregatorInternal(String name, CombineFn combiner) { - return null; - } - - } } From 33ed3238e2b3899cff061be3056c5cc29fc60a04 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 17:28:16 -0800 Subject: [PATCH 202/279] Moves DoFnAdapters to runners-core --- .../apex/translation/WindowBoundTranslator.java | 2 +- .../operators/ApexGroupByKeyOperator.java | 2 +- .../translation/operators/ApexParDoOperator.java | 2 +- .../org/apache/beam/runners/core}/DoFnAdapters.java | 8 ++++++-- .../beam/runners/core/SimpleOldDoFnRunner.java | 4 ++-- .../runners/core/GroupAlsoByWindowsProperties.java | 2 +- .../translation/functions/FlinkDoFnFunction.java | 2 +- .../functions/FlinkMultiOutputDoFnFunction.java | 2 +- .../functions/FlinkProcessContextBase.java | 2 +- .../wrappers/streaming/DoFnOperator.java | 2 +- .../beam/sdk/transforms/AggregatorRetriever.java | 13 +++++++++++-- .../org/apache/beam/sdk/transforms/OldDoFn.java | 2 +- .../org/apache/beam/sdk/transforms/NoOpOldDoFn.java | 2 +- 13 files changed, 29 insertions(+), 16 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/transforms => runners/core-java/src/main/java/org/apache/beam/runners/core}/DoFnAdapters.java (97%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java index 33b9269f9c43..ef049e19ce88 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; import org.apache.beam.runners.core.AssignWindowsDoFn; +import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index 48ac177ead7b..4af7ff0330e2 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -413,7 +413,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - protected Aggregator createAggregatorInternal( + public Aggregator createAggregatorInternal( String name, Combine.CombineFn combiner) { throw new UnsupportedOperationException(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java index 08f062de512b..1e769496651f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java @@ -37,6 +37,7 @@ import org.apache.beam.runners.apex.translation.utils.NoOpStepContext; import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; import org.apache.beam.runners.apex.translation.utils.ValueAndCoderKryoSerializable; +import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; @@ -48,7 +49,6 @@ import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java index 0a71faab2815..0f5624f56166 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java @@ -15,14 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.runners.core; import java.io.IOException; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.AggregatorRetriever; import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; @@ -69,7 +73,7 @@ private static class SimpleDoFnAdapter extends OldDoFn invoker; SimpleDoFnAdapter(DoFn fn) { - super(fn.aggregators); + super(AggregatorRetriever.getDelegatingAggregators(fn)); this.fn = fn; this.invoker = DoFnInvokers.invokerFor(fn); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 73286ade209b..10af29aa2bdf 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -323,7 +323,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - protected Aggregator createAggregatorInternal( + public Aggregator createAggregatorInternal( String name, CombineFn combiner) { checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null"); return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner); @@ -505,7 +505,7 @@ public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { } @Override - protected Aggregator + public Aggregator createAggregatorInternal( String name, CombineFn combiner) { return context.createAggregatorInternal(name, combiner); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java index 97b67c653c9c..ef01106fdf77 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java @@ -744,7 +744,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - protected Aggregator createAggregatorInternal( + public Aggregator createAggregatorInternal( String name, CombineFn combiner) { throw new UnsupportedOperationException(); } 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 ed200d58aa57..2a4a68e7b5bb 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,10 +18,10 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; +import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue; 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 7f6a4369ad85..a97bd46fc77b 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,10 +18,10 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; +import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java index 6afca38dcce4..53b98038adb9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java @@ -252,7 +252,7 @@ protected abstract void outputWithTimestampAndWindow( public abstract void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp); @Override - protected Aggregator + public Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { @SuppressWarnings("unchecked") SerializableFnAggregatorWrapper result = diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 6729aaa2ae38..87b15a7e08e6 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; @@ -40,7 +41,6 @@ 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.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java index ce47e22fda06..b1d3ead72ee2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java @@ -18,9 +18,10 @@ package org.apache.beam.sdk.transforms; import java.util.Collection; +import java.util.Map; /** - * An internal class for extracting {@link Aggregator Aggregators} from {@link OldDoFn DoFns}. + * An internal class for extracting {@link Aggregator Aggregators} from {@link DoFn DoFns}. */ public final class AggregatorRetriever { private AggregatorRetriever() { @@ -28,9 +29,17 @@ private AggregatorRetriever() { } /** - * Returns the {@link Aggregator Aggregators} created by the provided {@link OldDoFn}. + * Returns the {@link Aggregator Aggregators} created by the provided {@link DoFn}. */ public static Collection> getAggregators(DoFn fn) { return fn.getAggregators(); } + + /** + * Returns the {@link DelegatingAggregator delegating aggregators} created by the provided {@link + * DoFn}. + */ + public static Map> getDelegatingAggregators(DoFn fn) { + return fn.aggregators; + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index 0aef552f1c57..7b0453302a39 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -192,7 +192,7 @@ public abstract void sideOutputWithTimestamp( * context */ @Experimental(Kind.AGGREGATOR) - protected abstract Aggregator + public abstract Aggregator createAggregatorInternal(String name, CombineFn combiner); /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java index 504480b66cea..0db130db59dd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java @@ -63,7 +63,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { } @Override - protected Aggregator + public Aggregator createAggregatorInternal(String name, CombineFn combiner) { return null; } From 6b502fc111af266c7b1a0e6f7d473c36f57281a2 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 9 Dec 2016 17:29:41 -0800 Subject: [PATCH 203/279] Renames ParDo.getNewFn to getFn --- .../ParDoBoundMultiTranslator.java | 2 +- .../translation/ParDoBoundTranslator.java | 2 +- .../beam/runners/core/SplittableParDo.java | 4 ++-- .../runners/direct/ParDoEvaluatorFactory.java | 2 +- .../direct/ParDoMultiOverrideFactory.java | 2 +- .../ParDoSingleViaMultiOverrideFactory.java | 4 ++-- .../direct/StatefulParDoEvaluatorFactory.java | 4 ++-- .../FlinkBatchTransformTranslators.java | 4 ++-- .../FlinkStreamingTransformTranslators.java | 12 +++++----- .../dataflow/DataflowPipelineTranslator.java | 8 +++---- .../translation/TransformTranslator.java | 4 ++-- .../StreamingTransformTranslator.java | 4 ++-- .../beam/sdk/AggregatorPipelineExtractor.java | 4 ++-- .../org/apache/beam/sdk/transforms/ParDo.java | 22 +++++++++---------- .../sdk/AggregatorPipelineExtractorTest.java | 12 +++++----- 15 files changed, 45 insertions(+), 45 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java index 574ce8f067e8..bff76528a2d6 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundMultiTranslator.java @@ -53,7 +53,7 @@ class ParDoBoundMultiTranslator @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); if (signature.stateDeclarations().size() > 0) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java index de786289c9aa..3b6eb6ea34e8 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoBoundTranslator.java @@ -40,7 +40,7 @@ class ParDoBoundTranslator @Override public void translate(ParDo.Bound transform, TranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); if (signature.stateDeclarations().size() > 0) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 720db63137f7..f8d12ecb81f1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -104,7 +104,7 @@ public SplittableParDo(ParDo.BoundMulti parDo) { checkNotNull(parDo, "parDo must not be null"); this.parDo = parDo; checkArgument( - DoFnSignatures.getSignature(parDo.getNewFn().getClass()).processElement().isSplittable(), + DoFnSignatures.getSignature(parDo.getFn().getClass()).processElement().isSplittable(), "fn must be a splittable DoFn"); } @@ -114,7 +114,7 @@ public PCollectionTuple expand(PCollection input) { } private PCollectionTuple applyTyped(PCollection input) { - DoFn fn = parDo.getNewFn(); + DoFn fn = parDo.getFn(); Coder restrictionCoder = DoFnInvokers.invokerFor(fn) .invokeGetRestrictionCoder(input.getPipeline().getCoderRegistry()); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index ec5dc2c26400..b4684e34b266 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -65,7 +65,7 @@ public TransformEvaluator forApplication( application; ParDo.BoundMulti transform = parDoApplication.getTransform(); - final DoFn doFn = transform.getNewFn(); + final DoFn doFn = transform.getFn(); @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator evaluator = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 4e7914f310eb..4401434068d1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -46,7 +46,7 @@ class ParDoMultiOverrideFactory public PTransform, PCollectionTuple> override( ParDo.BoundMulti transform) { - DoFn fn = transform.getNewFn(); + DoFn fn = transform.getFn(); DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); if (signature.processElement().isSplittable()) { return new SplittableParDo(transform); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java index 10530bbc0285..5fcf49c257f5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java @@ -56,12 +56,12 @@ public PCollection expand(PCollection input) { PCollectionTuple outputs = input.apply( - ParDo.of(underlyingParDo.getNewFn()) + ParDo.of(underlyingParDo.getFn()) .withSideInputs(underlyingParDo.getSideInputs()) .withOutputTags(mainOutputTag, TupleTagList.empty())); PCollection output = outputs.get(mainOutputTag); - output.setTypeDescriptor(underlyingParDo.getNewFn().getOutputTypeDescriptor()); + output.setTypeDescriptor(underlyingParDo.getFn().getOutputTypeDescriptor()); return output; } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 1f3286c8e7ee..1f64d9ac7059 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -86,7 +86,7 @@ private TransformEvaluator>> createEvaluator( throws Exception { final DoFn, OutputT> doFn = - application.getTransform().getUnderlyingParDo().getNewFn(); + application.getTransform().getUnderlyingParDo().getFn(); final DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); // If the DoFn is stateful, schedule state clearing. @@ -141,7 +141,7 @@ public Runnable load( WindowingStrategy windowingStrategy = pc.getWindowingStrategy(); BoundedWindow window = transformOutputWindow.getWindow(); final DoFn doFn = - transformOutputWindow.getTransform().getTransform().getUnderlyingParDo().getNewFn(); + transformOutputWindow.getTransform().getTransform().getUnderlyingParDo().getFn(); final DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); final DirectStepContext stepContext = 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 497b2931f796..eb625b2a4be3 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 @@ -516,7 +516,7 @@ public void translateNode( ParDo.Bound transform, FlinkBatchTranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); DataSet> inputDataSet = @@ -562,7 +562,7 @@ private static class ParDoBoundMultiTranslatorBatch public void translateNode( ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); 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 42ef63098984..ffa6d166385e 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 @@ -338,7 +338,7 @@ public void translateNode( ParDo.Bound transform, FlinkStreamingTranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); WindowingStrategy windowingStrategy = @@ -358,7 +358,7 @@ public void translateNode( if (sideInputs.isEmpty()) { DoFnOperator> doFnOperator = new DoFnOperator<>( - transform.getNewFn(), + transform.getFn(), inputTypeInfo, new TupleTag("main output"), Collections.>emptyList(), @@ -381,7 +381,7 @@ public void translateNode( DoFnOperator> doFnOperator = new DoFnOperator<>( - transform.getNewFn(), + transform.getFn(), inputTypeInfo, new TupleTag("main output"), Collections.>emptyList(), @@ -490,7 +490,7 @@ public void translateNode( ParDo.BoundMulti transform, FlinkStreamingTranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); // we assume that the transformation does not change the windowing strategy. @@ -515,7 +515,7 @@ public void translateNode( if (sideInputs.isEmpty()) { DoFnOperator doFnOperator = new DoFnOperator<>( - transform.getNewFn(), + transform.getFn(), inputTypeInfo, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), @@ -542,7 +542,7 @@ public void translateNode( DoFnOperator doFnOperator = new DoFnOperator<>( - transform.getNewFn(), + transform.getFn(), inputTypeInfo, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index a56690c50ec6..8d2b0763be10 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -955,14 +955,14 @@ public void translate( private void translateMultiHelper( ParDo.BoundMulti transform, TranslationContext context) { - rejectStatefulDoFn(transform.getNewFn()); + rejectStatefulDoFn(transform.getFn()); context.addStep(transform, "ParallelDo"); translateInputs(context.getInput(transform), transform.getSideInputs(), context); BiMap> outputMap = translateOutputs(context.getOutput(transform), context); translateFn( - transform.getNewFn(), + transform.getFn(), context.getInput(transform).getWindowingStrategy(), transform.getSideInputs(), context.getInput(transform).getCoder(), @@ -985,13 +985,13 @@ public void translate( private void translateSingleHelper( ParDo.Bound transform, TranslationContext context) { - rejectStatefulDoFn(transform.getNewFn()); + rejectStatefulDoFn(transform.getFn()); context.addStep(transform, "ParallelDo"); translateInputs(context.getInput(transform), transform.getSideInputs(), context); long mainOutput = context.addOutput(context.getOutput(transform)); translateFn( - transform.getNewFn(), + transform.getFn(), context.getInput(transform).getWindowingStrategy(), transform.getSideInputs(), context.getInput(transform).getCoder(), diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index ac9189252fde..5dd6beb1ae45 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -227,7 +227,7 @@ private static TransformEvaluator return new TransformEvaluator>() { @Override public void evaluate(ParDo.Bound transform, EvaluationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); @SuppressWarnings("unchecked") JavaRDD> inRDD = @@ -250,7 +250,7 @@ public void evaluate(ParDo.Bound transform, EvaluationContext c return new TransformEvaluator>() { @Override public void evaluate(ParDo.BoundMulti transform, EvaluationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); @SuppressWarnings("unchecked") JavaRDD> inRDD = diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 27204edea596..070ccbb3c4b2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -344,7 +344,7 @@ private static TransformEvaluator @Override public void evaluate(final ParDo.Bound transform, final EvaluationContext context) { - final DoFn doFn = transform.getNewFn(); + final DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); final Map, KV, BroadcastHelper>> sideInputs = @@ -378,7 +378,7 @@ public JavaRDD> call(JavaRDD> rdd) @Override public void evaluate(final ParDo.BoundMulti transform, final EvaluationContext context) { - final DoFn doFn = transform.getNewFn(); + final DoFn doFn = transform.getFn(); rejectStateAndTimers(doFn); final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); final Map, KV, BroadcastHelper>> sideInputs = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java index ade5978b4eb0..c79f779d81bf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java @@ -70,10 +70,10 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { private Collection> getAggregators(PTransform transform) { if (transform != null) { if (transform instanceof ParDo.Bound) { - return AggregatorRetriever.getAggregators(((ParDo.Bound) transform).getNewFn()); + return AggregatorRetriever.getAggregators(((ParDo.Bound) transform).getFn()); } else if (transform instanceof ParDo.BoundMulti) { return AggregatorRetriever.getAggregators( - ((ParDo.BoundMulti) transform).getNewFn()); + ((ParDo.BoundMulti) transform).getFn()); } } return Collections.emptyList(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index d2149c054ef9..f897f8244630 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -745,7 +745,7 @@ public BoundMulti withOutputTags( @Override public PCollection expand(PCollection input) { checkArgument( - !isSplittable(getNewFn()), + !isSplittable(getFn()), "%s does not support Splittable DoFn", input.getPipeline().getOptions().getRunner().getName()); validateWindowType(input, fn); @@ -753,7 +753,7 @@ public PCollection expand(PCollection input) { input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setTypeDescriptor(getNewFn().getOutputTypeDescriptor()); + .setTypeDescriptor(getFn().getOutputTypeDescriptor()); } @Override @@ -761,14 +761,14 @@ public PCollection expand(PCollection input) { protected Coder getDefaultOutputCoder(PCollection input) throws CannotProvideCoderException { return input.getPipeline().getCoderRegistry().getDefaultCoder( - getNewFn().getOutputTypeDescriptor(), - getNewFn().getInputTypeDescriptor(), + getFn().getOutputTypeDescriptor(), + getFn().getInputTypeDescriptor(), ((PCollection) input).getCoder()); } @Override protected String getKindString() { - Class clazz = getNewFn().getClass(); + Class clazz = getFn().getClass(); if (clazz.isAnonymousClass()) { return "AnonymousParDo"; } else { @@ -789,7 +789,7 @@ public void populateDisplayData(Builder builder) { ParDo.populateDisplayData(builder, (HasDisplayData) fn, fnDisplayData); } - public DoFn getNewFn() { + public DoFn getFn() { return fn; } @@ -952,7 +952,7 @@ public BoundMulti withSideInputs( @Override public PCollectionTuple expand(PCollection input) { checkArgument( - !isSplittable(getNewFn()), + !isSplittable(getFn()), "%s does not support Splittable DoFn", input.getPipeline().getOptions().getRunner().getName()); validateWindowType(input, fn); @@ -965,7 +965,7 @@ public PCollectionTuple expand(PCollection input) { // The fn will likely be an instance of an anonymous subclass // such as DoFn { }, thus will have a high-fidelity // TypeDescriptor for the output type. - outputs.get(mainOutputTag).setTypeDescriptor(getNewFn().getOutputTypeDescriptor()); + outputs.get(mainOutputTag).setTypeDescriptor(getFn().getOutputTypeDescriptor()); return outputs; } @@ -984,13 +984,13 @@ public Coder getDefaultOutputCoder( Coder inputCoder = ((PCollection) input).getCoder(); return input.getPipeline().getCoderRegistry().getDefaultCoder( output.getTypeDescriptor(), - getNewFn().getInputTypeDescriptor(), + getFn().getInputTypeDescriptor(), inputCoder); } @Override protected String getKindString() { - Class clazz = getNewFn().getClass(); + Class clazz = getFn().getClass(); if (clazz.isAnonymousClass()) { return "AnonymousParMultiDo"; } else { @@ -1004,7 +1004,7 @@ public void populateDisplayData(Builder builder) { ParDo.populateDisplayData(builder, fn, fnDisplayData); } - public DoFn getNewFn() { + public DoFn getFn() { return fn; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java index c4e9b8a30519..1bf2c3df3d2f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java @@ -68,7 +68,7 @@ public void testGetAggregatorStepsWithParDoBoundExtractsSteps() { @SuppressWarnings("rawtypes") ParDo.Bound bound = mock(ParDo.Bound.class, "Bound"); AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); - when(bound.getNewFn()).thenReturn(fn); + when(bound.getFn()).thenReturn(fn); Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinIntegerFn()); @@ -96,7 +96,7 @@ public void testGetAggregatorStepsWithParDoBoundMultiExtractsSteps() { @SuppressWarnings("rawtypes") ParDo.BoundMulti bound = mock(ParDo.BoundMulti.class, "BoundMulti"); AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); - when(bound.getNewFn()).thenReturn(fn); + when(bound.getFn()).thenReturn(fn); Aggregator aggregatorOne = fn.addAggregator(new Max.MaxLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn()); @@ -126,8 +126,8 @@ public void testGetAggregatorStepsWithOneAggregatorInMultipleStepsAddsSteps() { @SuppressWarnings("rawtypes") ParDo.BoundMulti otherBound = mock(ParDo.BoundMulti.class, "otherBound"); AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); - when(bound.getNewFn()).thenReturn(fn); - when(otherBound.getNewFn()).thenReturn(fn); + when(bound.getFn()).thenReturn(fn); + when(otherBound.getFn()).thenReturn(fn); Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); Aggregator aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn()); @@ -162,7 +162,7 @@ public void testGetAggregatorStepsWithDifferentStepsAddsSteps() { AggregatorProvidingDoFn fn = new AggregatorProvidingDoFn<>(); Aggregator aggregatorOne = fn.addAggregator(new Sum.SumLongFn()); - when(bound.getNewFn()).thenReturn(fn); + when(bound.getFn()).thenReturn(fn); @SuppressWarnings("rawtypes") ParDo.BoundMulti otherBound = mock(ParDo.BoundMulti.class, "otherBound"); @@ -170,7 +170,7 @@ public void testGetAggregatorStepsWithDifferentStepsAddsSteps() { AggregatorProvidingDoFn otherFn = new AggregatorProvidingDoFn<>(); Aggregator aggregatorTwo = otherFn.addAggregator(new Sum.SumDoubleFn()); - when(otherBound.getNewFn()).thenReturn(otherFn); + when(otherBound.getFn()).thenReturn(otherFn); TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class); when(transformNode.getTransform()).thenReturn(bound); From ce75ba1ec5b4265bad9f3ff9b4b66818424b1375 Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 14 Dec 2016 11:23:39 -0800 Subject: [PATCH 204/279] Don't incorrectly log error in MetricsEnvironment Using getCurrentContainer() logs an error if metrics are not supported. This is because it acts as the common point of access for user code that reports metrics. It should not be used within setCurrentContainer(), because the first container being set will have a null previous-current-container, which will cause the error to be incorrectly logged. --- .../java/org/apache/beam/sdk/metrics/MetricsEnvironment.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java index 7c06cbf7150d..5d7cb0b1c093 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java @@ -56,7 +56,7 @@ public class MetricsEnvironment { */ @Nullable public static MetricsContainer setCurrentContainer(@Nullable MetricsContainer container) { - MetricsContainer previous = getCurrentContainer(); + MetricsContainer previous = CONTAINER_FOR_THREAD.get(); if (container == null) { CONTAINER_FOR_THREAD.remove(); } else { From e26f4075af6f2c990e23dc9f8fc8be2233652a9f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 16:02:23 -0800 Subject: [PATCH 205/279] Remove deprecated methods of InMemoryTimerInternals --- .../util/state/InMemoryTimerInternals.java | 26 ------------ .../state/InMemoryTimerInternalsTest.java | 40 +++++++------------ 2 files changed, 15 insertions(+), 51 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index 159b5830b642..44f901648ddc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -273,17 +273,6 @@ private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) { } } - /** Advances input watermark to the given value and fires event-time timers accordingly. - * - * @deprecated Use advanceInputWatermark without callback and fireEventTimers. - */ - @Deprecated - public void advanceInputWatermark( - TimerCallback timerCallback, Instant newInputWatermark) throws Exception { - advanceInputWatermark(newInputWatermark); - advanceAndFire(timerCallback, newInputWatermark, TimeDomain.EVENT_TIME); - } - /** Advances processing time to the given value and fires processing-time timers accordingly. * * @deprecated Use advanceProcessingTime without callback and fireProcessingTimers. @@ -295,21 +284,6 @@ public void advanceProcessingTime( advanceAndFire(timerCallback, newProcessingTime, TimeDomain.PROCESSING_TIME); } - /** - * Advances synchronized processing time to the given value and fires processing-time timers - * accordingly. - * - * @deprecated Use advanceInputWatermark without callback and fireSynchronizedProcessingTimers. - */ - @Deprecated - public void advanceSynchronizedProcessingTime( - TimerCallback timerCallback, Instant newSynchronizedProcessingTime) - throws Exception { - advanceSynchronizedProcessingTime(newSynchronizedProcessingTime); - advanceAndFire( - timerCallback, newSynchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); - } - @Deprecated private void advanceAndFire( TimerCallback timerCallback, Instant currentTime, TimeDomain domain) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java index 1e42864bf3ed..4a2763ccc76e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.util.state; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.mockito.Mockito.times; +import static org.junit.Assert.assertThat; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.joda.time.Instant; -import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.MockitoAnnotations; /** * Tests for {@link InMemoryTimerInternals}. @@ -40,14 +38,6 @@ public class InMemoryTimerInternalsTest { private static final StateNamespace NS1 = new StateNamespaceForTest("NS1"); - @Mock - private TimerCallback timerCallback; - - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - } - @Test public void testFiringTimers() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); @@ -85,24 +75,24 @@ public void testFiringTimersWithCallback() throws Exception { underTest.setTimer(processingTime1); underTest.setTimer(processingTime2); - underTest.advanceProcessingTime(timerCallback, new Instant(20)); - Mockito.verify(timerCallback).onTimer(processingTime1); - Mockito.verifyNoMoreInteractions(timerCallback); + underTest.advanceProcessingTime(new Instant(20)); + assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); // Advancing just a little shouldn't refire - underTest.advanceProcessingTime(timerCallback, new Instant(21)); - Mockito.verifyNoMoreInteractions(timerCallback); + underTest.advanceProcessingTime(new Instant(21)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); - // Adding the timer and advancing a little should refire + // Adding the timer and advancing a little should fire again underTest.setTimer(processingTime1); - underTest.advanceProcessingTime(timerCallback, new Instant(21)); - Mockito.verify(timerCallback, times(2)).onTimer(processingTime1); - Mockito.verifyNoMoreInteractions(timerCallback); + underTest.advanceProcessingTime(new Instant(21)); + assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); // And advancing the rest of the way should still have the other timer - underTest.advanceProcessingTime(timerCallback, new Instant(30)); - Mockito.verify(timerCallback).onTimer(processingTime2); - Mockito.verifyNoMoreInteractions(timerCallback); + underTest.advanceProcessingTime(new Instant(30)); + assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); } @Test From 5587e1cccb08839ef57a7d19b5dc3cf0a3725646 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 20:26:40 -0800 Subject: [PATCH 206/279] Delete deprecated TimerCallback --- .../util/state/InMemoryTimerInternals.java | 25 ------------ .../beam/sdk/util/state/TimerCallback.java | 38 ------------------- 2 files changed, 63 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index 44f901648ddc..44b44f06ead6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -272,29 +272,4 @@ private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) { return null; } } - - /** Advances processing time to the given value and fires processing-time timers accordingly. - * - * @deprecated Use advanceProcessingTime without callback and fireProcessingTimers. - */ - @Deprecated - public void advanceProcessingTime( - TimerCallback timerCallback, Instant newProcessingTime) throws Exception { - advanceProcessingTime(newProcessingTime); - advanceAndFire(timerCallback, newProcessingTime, TimeDomain.PROCESSING_TIME); - } - - @Deprecated - private void advanceAndFire( - TimerCallback timerCallback, Instant currentTime, TimeDomain domain) - throws Exception { - checkNotNull(timerCallback); - TimerData timer; - while ((timer = removeNextTimer(currentTime, domain)) != null) { - WindowTracing.trace( - "{}.advanceAndFire: firing {} at {}", - getClass().getSimpleName(), timer, currentTime); - timerCallback.onTimer(timer); - } - } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java deleted file mode 100644 index 83791d6fade0..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerCallback.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.state; - -import org.apache.beam.sdk.util.TimerInternals; - -/** - * A callback that processes a {@link TimerInternals.TimerData TimerData}. - * - * @deprecated Use InMemoryTimerInternals advance and remove methods instead of callback. - */ -@Deprecated -public interface TimerCallback { - /** Processes the {@link TimerInternals.TimerData TimerData}. */ - void onTimer(TimerInternals.TimerData timer) throws Exception; - - TimerCallback NO_OP = new TimerCallback() { - @Override - public void onTimer(TimerInternals.TimerData timer) throws Exception { - // Nothing - } - }; -} From ec0bf7b4023ff75f4ec6723d2e77ed507eb57c51 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 20:45:56 -0800 Subject: [PATCH 207/279] Move InMemoryTimerInternals to runners-core --- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 1 - .../runners/core}/InMemoryTimerInternals.java | 3 +- .../core}/InMemoryTimerInternalsTest.java | 4 ++- .../beam/runners/core/ReduceFnTester.java | 1 - .../runners/core/SplittableParDoTest.java | 16 +++++++-- .../triggers/TriggerStateMachineTester.java | 2 +- .../translation/SparkGroupAlsoByWindowFn.java | 2 +- .../beam/sdk/transforms/DoFnTester.java | 36 ------------------- 8 files changed, 20 insertions(+), 45 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util/state => runners/core-java/src/main/java/org/apache/beam/runners/core}/InMemoryTimerInternals.java (99%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util/state => runners/core-java/src/test/java/org/apache/beam/runners/core}/InMemoryTimerInternalsTest.java (97%) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index 918919170ec6..efcd771d01b9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java similarity index 99% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index 44b44f06ead6..b22fcb3d2c0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -29,6 +29,7 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowTracing; +import org.apache.beam.sdk.util.state.StateNamespace; import org.joda.time.Instant; /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java similarity index 97% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java index 4a2763ccc76e..2caa8742dea1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.runners.core; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -25,6 +25,8 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaceForTest; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index db0cf9186a7d..890195a0d3c1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -73,7 +73,6 @@ import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index cf96b660bea6..41d419baa8b2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -28,6 +28,7 @@ import static org.junit.Assert.assertTrue; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -190,6 +191,8 @@ private static class ProcessFnTester< tester; private Instant currentProcessingTime; + private InMemoryTimerInternals timerInternals; + ProcessFnTester( Instant currentProcessingTime, DoFn fn, @@ -200,6 +203,7 @@ private static class ProcessFnTester< new SplittableParDo.ProcessFn<>( fn, inputCoder, restrictionCoder, IntervalWindow.getCoder()); this.tester = DoFnTester.of(processFn); + this.timerInternals = new InMemoryTimerInternals(); processFn.setStateInternalsFactory( new StateInternalsFactory() { @Override @@ -211,7 +215,7 @@ public StateInternals stateInternalsForKey(String key) { new TimerInternalsFactory() { @Override public TimerInternals timerInternalsForKey(String key) { - return tester.getTimerInternals(); + return timerInternals; } }); processFn.setOutputWindowedValue( @@ -247,7 +251,7 @@ public void sideOutputWindowedValue( // through the state/timer/output callbacks. this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE); this.tester.startBundle(); - this.tester.advanceProcessingTime(currentProcessingTime); + timerInternals.advanceProcessingTime(currentProcessingTime); this.currentProcessingTime = currentProcessingTime; } @@ -285,7 +289,13 @@ void startElement( */ boolean advanceProcessingTimeBy(Duration duration) throws Exception { currentProcessingTime = currentProcessingTime.plus(duration); - List timers = tester.advanceProcessingTime(currentProcessingTime); + timerInternals.advanceProcessingTime(currentProcessingTime); + + List timers = new ArrayList<>(); + TimerInternals.TimerData nextTimer; + while ((nextTimer = timerInternals.removeNextProcessingTimer()) != null) { + timers.add(nextTimer); + } if (timers.isEmpty()) { return false; } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index ed5ce9c960fb..89694053831b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -46,7 +47,6 @@ import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java index 87d3f5059039..5432d58994af 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.List; import org.apache.beam.runners.core.GroupAlsoByWindowsDoFn; +import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; import org.apache.beam.runners.core.SystemReduceFn; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 527d529cda4b..b2c3fd5ed2e9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -46,11 +46,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.InMemoryStateInternals; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; @@ -142,10 +140,6 @@ public StateInternals getStateInternals() { return (StateInternals) stateInternals; } - public TimerInternals getTimerInternals() { - return timerInternals; - } - /** * When a {@link DoFnTester} should clone the {@link DoFn} under test and how it should manage * the lifecycle of the {@link DoFn}. @@ -232,7 +226,6 @@ public void startBundle() throws Exception { context.setupDelegateAggregators(); // State and timer internals are per-bundle. stateInternals = InMemoryStateInternals.forKey(new Object()); - timerInternals = new InMemoryTimerInternals(); try { fnInvoker.invokeStartBundle(context); } catch (UserCodeException e) { @@ -535,34 +528,6 @@ public AggregateT getAggregatorValue(Aggregator agg) return extractAggregatorValue(agg.getName(), agg.getCombineFn()); } - public List advanceInputWatermark(Instant newWatermark) { - try { - timerInternals.advanceInputWatermark(newWatermark); - final List firedTimers = new ArrayList<>(); - TimerInternals.TimerData timer; - while ((timer = timerInternals.removeNextEventTimer()) != null) { - firedTimers.add(timer); - } - return firedTimers; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public List advanceProcessingTime(Instant newProcessingTime) { - try { - timerInternals.advanceProcessingTime(newProcessingTime); - final List firedTimers = new ArrayList<>(); - TimerInternals.TimerData timer; - while ((timer = timerInternals.removeNextProcessingTimer()) != null) { - firedTimers.add(timer); - } - return firedTimers; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - private AggregateT extractAggregatorValue( String name, CombineFn combiner) { @SuppressWarnings("unchecked") @@ -807,7 +772,6 @@ private enum State { private Map, List>> outputs; private InMemoryStateInternals stateInternals; - private InMemoryTimerInternals timerInternals; /** The state of processing of the {@link DoFn} under test. */ private State state = State.UNINITIALIZED; From 55f0495583312c9c0dea620d6a4e85193e97f255 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 21:06:14 -0800 Subject: [PATCH 208/279] [BEAM-362] Port runners to runners-core AggregatoryFactory --- .../apex/translation/operators/ApexParDoOperator.java | 2 +- .../main/java/org/apache/beam/runners/core/DoFnRunners.java | 1 - .../java/org/apache/beam/runners/core/SimpleDoFnRunner.java | 1 - .../org/apache/beam/runners/core/SimpleOldDoFnRunner.java | 1 - .../org/apache/beam/runners/direct/AggregatorContainer.java | 2 +- .../flink/translation/wrappers/streaming/DoFnOperator.java | 3 ++- .../beam/runners/spark/aggregators/SparkAggregators.java | 5 +++-- 7 files changed, 7 insertions(+), 8 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java index 1e769496651f..4538fb5a052b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java @@ -37,6 +37,7 @@ import org.apache.beam.runners.apex.translation.utils.NoOpStepContext; import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; import org.apache.beam.runners.apex.translation.utils.ValueAndCoderKryoSerializable; +import org.apache.beam.runners.core.AggregatorFactory; import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -46,7 +47,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index da16573c3afa..0e4bf75ba0b4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -22,7 +22,6 @@ import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 041cdde69292..d504b403002c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -30,7 +30,6 @@ 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.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Context; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 10af29aa2bdf..7d9320062f27 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -28,7 +28,6 @@ import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java index e86bc3ebaf28..c7fa4df7cd8a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java @@ -27,8 +27,8 @@ import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; +import org.apache.beam.runners.core.AggregatorFactory; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.util.ExecutionContext; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 87b15a7e08e6..001e3b68ce0c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.AggregatorFactory; import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -192,7 +193,7 @@ public void open() throws Exception { currentInputWatermark = Long.MIN_VALUE; currentOutputWatermark = currentInputWatermark; - Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() { + AggregatorFactory aggregatorFactory = new AggregatorFactory() { @Override public Aggregator createAggregatorForDoFn( Class fnClass, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java index 657264f377a6..17d584481bc2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import java.util.Collection; import java.util.Map; +import org.apache.beam.runners.core.AggregatorFactory; import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.transforms.Aggregator; @@ -99,9 +100,9 @@ public static T valueOf(final String name, } /** - * An implementation of {@link Aggregator.AggregatorFactory} for the SparkRunner. + * An implementation of {@link AggregatorFactory} for the SparkRunner. */ - public static class Factory implements Aggregator.AggregatorFactory { + public static class Factory implements AggregatorFactory { private final SparkRuntimeContext runtimeContext; private final Accumulator accumulator; From 235027b9fc6e322c469b099d168e60bf72a567db Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 15 Dec 2016 13:50:39 -0800 Subject: [PATCH 209/279] [BEAM-450] Shade modules to separate paths --- runners/core-java/pom.xml | 4 ++-- runners/google-cloud-dataflow-java/pom.xml | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml index b5c610b97a21..704aeaf91f06 100644 --- a/runners/core-java/pom.xml +++ b/runners/core-java/pom.xml @@ -90,11 +90,11 @@ the second relocation. --> com.google.common - org.apache.beam.sdk.repackaged.com.google.common + org.apache.beam.runners.core.repackaged.com.google.common com.google.thirdparty - org.apache.beam.sdk.repackaged.com.google.thirdparty + org.apache.beam.runners.core.repackaged.com.google.thirdparty diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 77187d68cd4f..46ac7ef11c2d 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -133,15 +133,15 @@ com.google.common.**.testing.* - org.apache.beam.sdk.repackaged.com.google.common + org.apache.beam.runners.dataflow.repackaged.com.google.common com.google.thirdparty - org.apache.beam.sdk.repackaged.com.google.thirdparty + org.apache.beam.runners.dataflow.repackaged.com.google.thirdparty com.google.cloud.bigtable - org.apache.beam.sdk.repackaged.com.google.cloud.bigtable + org.apache.beam.runners.dataflow.repackaged.com.google.cloud.bigtable com.google.cloud.bigtable.config.BigtableOptions* com.google.cloud.bigtable.config.CredentialOptions* From b72333331fd4d79063b2feae1ac59d5c54f2b337 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 16 Dec 2016 08:23:22 -0800 Subject: [PATCH 210/279] [BEAM-1108] Remove outdated language about experimental autoscaling --- .../beam/examples/complete/TopWikipediaSessions.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index df7f81e23e65..8e0b815bf260 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -62,15 +62,6 @@ * *

      The default input is {@code gs://apache-beam-samples/wikipedia_edits/*.json} and can be * overridden with {@code --input}. - * - *

      The input for this example is large enough that it's a good place to enable (experimental) - * autoscaling: - *

      {@code
      - *   --autoscalingAlgorithm=BASIC
      - *   --maxNumWorkers=20
      - * }
      - * 
      - * This will automatically scale the number of workers up over time until the job completes. */ public class TopWikipediaSessions { private static final String EXPORTED_WIKI_TABLE = From cc662d6fd64a8d00cb3f55df9cd04e0b2a4a3bbf Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 16 Dec 2016 13:25:23 -0800 Subject: [PATCH 211/279] Update Dataflow worker to beam-master-20161216 --- .../org/apache/beam/runners/dataflow/dataflow.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index d43881dfe304..27a518f02713 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -18,6 +18,6 @@ environment.major.version=6 -worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161212 +worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161216 -worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161212 +worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161216 From bcf02986df5d7831bb6fbe4c304bef6857e395f3 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 12 Dec 2016 20:18:02 -0800 Subject: [PATCH 212/279] Move ActiveWindowSet and implementations to runners-core --- .../org/apache/beam/runners/core}/ActiveWindowSet.java | 2 +- .../apache/beam/runners/core}/MergingActiveWindowSet.java | 2 +- .../beam/runners/core}/NonMergingActiveWindowSet.java | 2 +- .../apache/beam/runners/core/ReduceFnContextFactory.java | 1 - .../java/org/apache/beam/runners/core/ReduceFnRunner.java | 3 --- .../core/triggers/TriggerStateMachineContextFactory.java | 2 +- .../beam/runners/core}/MergingActiveWindowSetTest.java | 2 +- .../runners/core/triggers/TriggerStateMachineTester.java | 8 ++++---- 8 files changed, 9 insertions(+), 13 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/ActiveWindowSet.java (99%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/MergingActiveWindowSet.java (99%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/NonMergingActiveWindowSet.java (98%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util => runners/core-java/src/test/java/org/apache/beam/runners/core}/MergingActiveWindowSetTest.java (99%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ActiveWindowSet.java similarity index 99% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ActiveWindowSet.java index 2e0af29732a1..79d1f3f9927e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ActiveWindowSet.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.annotations.VisibleForTesting; import java.util.Collection; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java similarity index 99% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java index 066579b5045e..720377adacd9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonMergingActiveWindowSet.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/NonMergingActiveWindowSet.java index 99d591bb00b0..fec6c4531d3f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonMergingActiveWindowSet.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index c71897dadc98..eae1a8b3b547 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.ActiveWindowSet; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index 6f7bbcf673f7..96e76b7559c9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -50,9 +50,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.ActiveWindowSet; -import org.apache.beam.sdk.util.MergingActiveWindowSet; -import org.apache.beam.sdk.util.NonMergingActiveWindowSet; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java index 1c06e8dbaa9d..e3df4ee278d9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java @@ -24,12 +24,12 @@ import java.util.Collection; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.runners.core.ActiveWindowSet; import org.apache.beam.runners.core.triggers.TriggerStateMachine.MergingTriggerInfo; import org.apache.beam.runners.core.triggers.TriggerStateMachine.TriggerInfo; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.ActiveWindowSet; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.state.MergingStateAccessor; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java similarity index 99% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java index 676a25ab42a0..a4928e36d32a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 89694053831b..2a626d4311d5 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -32,15 +32,15 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.runners.core.ActiveWindowSet; +import org.apache.beam.runners.core.ActiveWindowSet.MergeCallback; import org.apache.beam.runners.core.InMemoryTimerInternals; +import org.apache.beam.runners.core.MergingActiveWindowSet; +import org.apache.beam.runners.core.NonMergingActiveWindowSet; 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.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.ActiveWindowSet; -import org.apache.beam.sdk.util.ActiveWindowSet.MergeCallback; -import org.apache.beam.sdk.util.MergingActiveWindowSet; -import org.apache.beam.sdk.util.NonMergingActiveWindowSet; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.Timers; From 3f227a0ad18f425767e89f88d8a1c9fdcebdd80c Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 5 Dec 2016 16:01:57 -0800 Subject: [PATCH 213/279] Add PTransformOverrideFactory to the Core SDK This migrates PTransformOverrideFactory from the DirectRunner to the Core SDK, as part of BEAM-646. Add getOriginalToReplacements to provide a mapping from the original outputs to replaced outputs. This enables all replaced nodes to be rewired to output the original output. Migrate all DirectRunner Override Factories to the new PTransformOverrideFactory. --- ...tGBKIntoKeyedWorkItemsOverrideFactory.java | 7 +++--- .../DirectGroupByKeyOverrideFactory.java | 3 ++- .../beam/runners/direct/DirectRunner.java | 7 ++++-- .../direct/ParDoMultiOverrideFactory.java | 9 ++++---- .../ParDoSingleViaMultiOverrideFactory.java | 11 +++++----- .../direct/TestStreamEvaluatorFactory.java | 5 ++++- .../runners/direct/ViewEvaluatorFactory.java | 4 +++- .../direct/WriteWithShardingFactory.java | 6 +++-- .../direct/WriteWithShardingFactoryTest.java | 4 ++-- .../beam/sdk/annotations/Experimental.java | 5 ++++- .../runners}/PTransformOverrideFactory.java | 22 ++++++++++++------- 11 files changed, 53 insertions(+), 30 deletions(-) rename {runners/direct-java/src/main/java/org/apache/beam/runners/direct => sdks/java/core/src/main/java/org/apache/beam/sdk/runners}/PTransformOverrideFactory.java (61%) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java index 1fa059c0800d..ab4c114965c7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java @@ -18,7 +18,8 @@ package org.apache.beam.runners.direct; import org.apache.beam.runners.core.KeyedWorkItem; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDo.GBKIntoKeyedWorkItems; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -30,10 +31,10 @@ class DirectGBKIntoKeyedWorkItemsOverrideFactory implements PTransformOverrideFactory< PCollection>, PCollection>, - SplittableParDo.GBKIntoKeyedWorkItems> { + GBKIntoKeyedWorkItems> { @Override public PTransform>, PCollection>> - override(SplittableParDo.GBKIntoKeyedWorkItems transform) { + getReplacementTransform(GBKIntoKeyedWorkItems transform) { return new DirectGroupByKey.DirectGroupByKeyOnly<>(); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java index 9acf5e9151b1..7cf325686a20 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; @@ -27,7 +28,7 @@ final class DirectGroupByKeyOverrideFactory implements PTransformOverrideFactory< PCollection>, PCollection>>, GroupByKey> { @Override - public PTransform>, PCollection>>> override( + public PTransform>, PCollection>>> getReplacementTransform( GroupByKey transform) { return new DirectGroupByKey<>(transform); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 2f84356d6e91..78163c0d7e23 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -46,6 +46,7 @@ import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.Aggregator; @@ -284,9 +285,11 @@ void setClockSupplier(Supplier supplier) { @Override public OutputT apply( PTransform transform, InputT input) { - PTransformOverrideFactory overrideFactory = defaultTransformOverrides.get(transform.getClass()); + PTransformOverrideFactory> overrideFactory = + defaultTransformOverrides.get(transform.getClass()); if (overrideFactory != null) { - PTransform customTransform = overrideFactory.override(transform); + PTransform customTransform = + overrideFactory.getReplacementTransform(transform); if (customTransform != transform) { return Pipeline.applyTransform(transform.getName(), input, customTransform); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 4401434068d1..c5bc0698cad7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -20,10 +20,12 @@ import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.DoFn; 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.ParDo.BoundMulti; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.values.KV; @@ -39,12 +41,11 @@ */ class ParDoMultiOverrideFactory implements PTransformOverrideFactory< - PCollection, PCollectionTuple, ParDo.BoundMulti> { - + PCollection, PCollectionTuple, BoundMulti> { @Override @SuppressWarnings("unchecked") - public PTransform, PCollectionTuple> override( - ParDo.BoundMulti transform) { + public PTransform, PCollectionTuple> getReplacementTransform( + BoundMulti transform) { DoFn fn = transform.getFn(); DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java index 5fcf49c257f5..3ae338285e56 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleViaMultiOverrideFactory.java @@ -17,8 +17,10 @@ */ package org.apache.beam.runners.direct; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.ParDo.Bound; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; @@ -30,12 +32,11 @@ */ class ParDoSingleViaMultiOverrideFactory implements PTransformOverrideFactory< - PCollection, PCollection, ParDo.Bound> { + PCollection, PCollection, Bound>{ @Override - @SuppressWarnings("unchecked") - public PTransform, PCollection> override( - ParDo.Bound transform) { - return new ParDoSingleViaMulti(transform); + public PTransform, PCollection> getReplacementTransform( + Bound transform) { + return new ParDoSingleViaMulti<>(transform); } static class ParDoSingleViaMulti diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 3601dbc3d3eb..6ba65bf008e7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -30,6 +30,7 @@ import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.TestStream.ElementEvent; @@ -157,8 +158,10 @@ public Clock get() { static class DirectTestStreamFactory implements PTransformOverrideFactory, TestStream> { + @Override - public PTransform> override(TestStream transform) { + public PTransform> getReplacementTransform( + TestStream transform) { return new DirectTestStream<>(transform); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index 460b1c2b694a..96a18d77f468 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -24,6 +24,7 @@ import org.apache.beam.runners.direct.StepTransformResult.Builder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; @@ -96,8 +97,9 @@ public TransformResult> finishBundle() { public static class ViewOverrideFactory implements PTransformOverrideFactory< PCollection, PCollectionView, CreatePCollectionView> { + @Override - public PTransform, PCollectionView> override( + public PTransform, PCollectionView> getReplacementTransform( CreatePCollectionView transform) { return new DirectCreatePCollectionView<>(transform); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index 3c8833764954..fd1c1751e54c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -47,11 +47,13 @@ * of shards is the log base 10 of the number of input records, with up to 2 additional shards. */ class WriteWithShardingFactory - implements PTransformOverrideFactory, PDone, Write.Bound> { + implements org.apache.beam.sdk.runners.PTransformOverrideFactory< + PCollection, PDone, Write.Bound> { static final int MAX_RANDOM_EXTRA_SHARDS = 3; @Override - public PTransform, PDone> override(Write.Bound transform) { + public PTransform, PDone> getReplacementTransform( + Bound transform) { if (transform.getNumShards() == 0) { return new DynamicallyReshardedWrite<>(transform); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index 1ff5de2c18bc..aeb75ed88968 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -121,13 +121,13 @@ public void dynamicallyReshardedWrite() throws Exception { public void withShardingSpecifiesOriginalTransform() { Write.Bound original = Write.to(new TestSink()).withNumShards(3); - assertThat(factory.override(original), equalTo((Object) original)); + assertThat(factory.getReplacementTransform(original), equalTo((Object) original)); } @Test public void withNoShardingSpecifiedReturnsNewTransform() { Write.Bound original = Write.to(new TestSink()); - assertThat(factory.override(original), not(equalTo((Object) original))); + assertThat(factory.getReplacementTransform(original), not(equalTo((Object) original))); } @Test diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java index c55cd5e38605..265965967889 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java @@ -85,6 +85,9 @@ enum Kind { SPLITTABLE_DO_FN, /** Metrics-related experimental APIs. */ - METRICS + METRICS, + + /** Experimental runner APIs. Should not be used by pipeline authors. */ + CORE_RUNNERS_ONLY } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java similarity index 61% rename from runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java index 8db6e9b358e5..f6e90e203042 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PTransformOverrideFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java @@ -14,22 +14,28 @@ * 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.direct; +package org.apache.beam.sdk.runners; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -interface PTransformOverrideFactory< +/** + * Produces {@link PipelineRunner}-specific overrides of {@link PTransform PTransforms}, and + * provides mappings between original and replacement outputs. + */ +@Experimental(Kind.CORE_RUNNERS_ONLY) +public interface PTransformOverrideFactory< InputT extends PInput, OutputT extends POutput, - TransformT extends PTransform> { + TransformT extends PTransform> { /** - * Create a {@link PTransform} override for the provided {@link PTransform} if applicable. - * Otherwise, return the input {@link PTransform}. - * - *

      The returned PTransform must be semantically equivalent to the input {@link PTransform}. + * Returns a {@link PTransform} that produces equivalent output to the provided transform. */ - PTransform override(TransformT transform); + PTransform getReplacementTransform(TransformT transform); } From cfd1633a441f98d100bd995bd811b6b8fa2cd62d Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 14 Dec 2016 15:46:50 -0800 Subject: [PATCH 214/279] Make {Metric,Counter,Distribution}Cell public These classes may be useful to hold on to within a runner where the step context is known to be unchanging and there may be a desire to avoid the extra indirection of using {Counter,Distribution} directly. --- .../org/apache/beam/sdk/metrics/CounterCell.java | 10 +++++++++- .../apache/beam/sdk/metrics/DistributionCell.java | 15 +++++++++++++-- .../org/apache/beam/sdk/metrics/MetricCell.java | 2 +- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java index bb6583329a80..93700e6efe50 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/CounterCell.java @@ -26,13 +26,21 @@ * *

      This class generally shouldn't be used directly. The only exception is within a runner where * a counter is being reported for a specific step (rather than the counter in the current context). + * In that case retrieving the underlying cell and reporting directly to it avoids a step of + * indirection. */ @Experimental(Kind.METRICS) -class CounterCell implements MetricCell, Counter { +public class CounterCell implements MetricCell, Counter { private final DirtyState dirty = new DirtyState(); private final AtomicLong value = new AtomicLong(); + /** + * Package-visibility because all {@link CounterCell CounterCells} should be created by + * {@link MetricsContainer#getCounter(MetricName)}. + */ + CounterCell() {} + /** Increment the counter by the given amount. */ private void add(long n) { value.addAndGet(n); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java index f0074a943701..e09515828680 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java @@ -23,14 +23,25 @@ /** * Tracks the current value (and delta) for a Distribution metric. + * + *

      This class generally shouldn't be used directly. The only exception is within a runner where + * a distribution is being reported for a specific step (rather than the distribution in the current + * context). In that case retrieving the underlying cell and reporting directly to it avoids a step + * of indirection. */ @Experimental(Kind.METRICS) -class DistributionCell implements MetricCell, Distribution { +public class DistributionCell implements MetricCell, Distribution { private final DirtyState dirty = new DirtyState(); private final AtomicReference value = new AtomicReference(DistributionData.EMPTY); + /** + * Package-visibility because all {@link DistributionCell DistributionCells} should be created by + * {@link MetricsContainer#getDistribution(MetricName)}. + */ + DistributionCell() {} + /** Increment the counter by the given amount. */ @Override public void update(long n) { @@ -55,4 +66,4 @@ public DistributionData getCumulative() { public Distribution getInterface() { return this; } -} +} \ No newline at end of file diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java index 211b2dd691ca..7cf9710c577e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricCell.java @@ -28,7 +28,7 @@ * @param The type of metric data stored (and extracted) from this cell. */ @Experimental(Kind.METRICS) -interface MetricCell { +public interface MetricCell { /** * Return the {@link DirtyState} tracking whether this metric cell contains uncommitted changes. From 92dbab8ecf2761893759c494d33aa8583e919a6f Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 16 Dec 2016 15:56:00 -0800 Subject: [PATCH 215/279] checkstyle: missed newline in DistributionCell --- .../java/org/apache/beam/sdk/metrics/DistributionCell.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java index e09515828680..7f684a86b405 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/DistributionCell.java @@ -66,4 +66,5 @@ public DistributionData getCumulative() { public Distribution getInterface() { return this; } -} \ No newline at end of file +} + From a12fd8c580d3b1ea46c5be951f39046bfa0dacf3 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 16 Dec 2016 15:26:28 -0800 Subject: [PATCH 216/279] Revert "Removes code for wrapping DoFn as an OldDoFn" This reverts commit a22de15012c51e8b7e31143021f0a298e093bf51. --- .../beam/runners/core/DoFnAdapters.java | 150 +++++++++ .../apache/beam/sdk/transforms/OldDoFn.java | 295 +++++++++++++++++- .../sdk/transforms/reflect/DoFnInvokers.java | 141 ++++++++- .../transforms/reflect/DoFnInvokersTest.java | 36 +++ 4 files changed, 611 insertions(+), 11 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java index 0f5624f56166..a4002daf9028 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.core; import java.io.IOException; +import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.AggregatorRetriever; @@ -39,6 +41,7 @@ import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; @@ -53,6 +56,18 @@ public class DoFnAdapters { /** Should not be instantiated. */ private DoFnAdapters() {} + /** + * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the + * original {@link DoFn}, otherwise returns {@code fn.getClass()}. + */ + public static Class getDoFnClass(OldDoFn fn) { + if (fn instanceof SimpleDoFnAdapter) { + return ((SimpleDoFnAdapter) fn).fn.getClass(); + } else { + return fn.getClass(); + } + } + /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */ @SuppressWarnings({"unchecked", "rawtypes"}) public static OldDoFn toOldDoFn(DoFn fn) { @@ -64,6 +79,126 @@ public static OldDoFn toOldDoFn(DoFn OldDoFn.ProcessContext adaptProcessContext( + OldDoFn fn, + final DoFn.ProcessContext c, + final DoFnInvoker.ArgumentProvider extra) { + return fn.new ProcessContext() { + @Override + public InputT element() { + return c.element(); + } + + @Override + public T sideInput(PCollectionView view) { + return c.sideInput(view); + } + + @Override + public Instant timestamp() { + return c.timestamp(); + } + + @Override + public BoundedWindow window() { + return extra.window(); + } + + @Override + public PaneInfo pane() { + return c.pane(); + } + + @Override + public WindowingInternals windowingInternals() { + return extra.windowingInternals(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return c.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + c.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + c.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + c.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + c.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { + return c.createAggregator(name, combiner); + } + }; + } + + /** Creates a {@link OldDoFn.ProcessContext} from a {@link DoFn.ProcessContext}. */ + public static OldDoFn.Context adaptContext( + OldDoFn fn, + final DoFn.Context c) { + return fn.new Context() { + @Override + public PipelineOptions getPipelineOptions() { + return c.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + c.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + c.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + c.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + c.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { + return c.createAggregator(name, combiner); + } + }; + } + + /** + * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise, + * returns {@code null}. + */ + @Nullable + public static DoFn getDoFn(OldDoFn fn) { + if (fn instanceof SimpleDoFnAdapter) { + return ((SimpleDoFnAdapter) fn).fn; + } else { + return null; + } + } + /** * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link * OldDoFn}. @@ -105,6 +240,21 @@ public void processElement(ProcessContext c) throws Exception { invoker.invokeProcessElement(adapter); } + @Override + protected TypeDescriptor getInputTypeDescriptor() { + return fn.getInputTypeDescriptor(); + } + + @Override + protected TypeDescriptor getOutputTypeDescriptor() { + return fn.getOutputTypeDescriptor(); + } + + @Override + Collection> getAggregators() { + return fn.getAggregators(); + } + @Override public Duration getAllowedTimestampSkew() { return fn.getAllowedTimestampSkew(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index 7b0453302a39..d1bb42bec7f6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; @@ -70,6 +71,21 @@ */ @Deprecated public abstract class OldDoFn implements Serializable, HasDisplayData { + + public DoFn toDoFn() { + DoFn doFn = DoFnAdapters.getDoFn(this); + if (doFn != null) { + return doFn; + } + if (this instanceof RequiresWindowAccess) { + // No parameters as it just accesses `this` + return new AdaptedRequiresWindowAccessDoFn(); + } else { + // No parameters as it just accesses `this` + return new AdaptedDoFn(); + } + } + /** * Information accessible to all methods in this {@code OldDoFn}. * Used primarily to output elements. @@ -318,7 +334,7 @@ public OldDoFn() { this(new HashMap>()); } - public OldDoFn(Map> aggregators) { + OldDoFn(Map> aggregators) { this.aggregators = aggregators; } @@ -402,6 +418,32 @@ public void populateDisplayData(DisplayData.Builder builder) { ///////////////////////////////////////////////////////////////////////////// + /** + * Returns a {@link TypeDescriptor} capturing what is known statically + * about the input type of this {@code OldDoFn} instance's most-derived + * class. + * + *

      See {@link #getOutputTypeDescriptor} for more discussion. + */ + protected TypeDescriptor getInputTypeDescriptor() { + return new TypeDescriptor(getClass()) {}; + } + + /** + * Returns a {@link TypeDescriptor} capturing what is known statically + * about the output type of this {@code OldDoFn} instance's + * most-derived class. + * + *

      In the normal case of a concrete {@code OldDoFn} subclass with + * no generic type parameters of its own (including anonymous inner + * classes), this will be a complete non-generic type, which is good + * for choosing a default output {@code Coder} for the output + * {@code PCollection}. + */ + protected TypeDescriptor getOutputTypeDescriptor() { + return new TypeDescriptor(getClass()) {}; + } + /** * Returns an {@link Aggregator} with aggregation logic specified by the * {@link CombineFn} argument. The name provided must be unique across @@ -462,4 +504,255 @@ protected final Aggregator createAggregator(St Collection> getAggregators() { return Collections.>unmodifiableCollection(aggregators.values()); } + + /** + * A {@link Context} for an {@link OldDoFn} via a context for a proper {@link DoFn}. + */ + private class AdaptedContext extends Context { + + private final DoFn.Context newContext; + + public AdaptedContext( + DoFn.Context newContext) { + this.newContext = newContext; + super.setupDelegateAggregators(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return newContext.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + newContext.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + newContext.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + newContext.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + newContext.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { + return newContext.createAggregator(name, combiner); + } + } + + /** + * A {@link ProcessContext} for an {@link OldDoFn} via a context for a proper {@link DoFn}. + */ + private class AdaptedProcessContext extends ProcessContext { + + private final DoFn.ProcessContext newContext; + + public AdaptedProcessContext(DoFn.ProcessContext newContext) { + this.newContext = newContext; + } + + @Override + public InputT element() { + return newContext.element(); + } + + @Override + public T sideInput(PCollectionView view) { + return newContext.sideInput(view); + } + + @Override + public Instant timestamp() { + return newContext.timestamp(); + } + + @Override + public BoundedWindow window() { + throw new UnsupportedOperationException(String.format( + "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s", + OldDoFn.class.getSimpleName(), + OldDoFn.ProcessContext.class.getSimpleName(), + OldDoFn.class.getSimpleName(), + DoFn.class.getSimpleName())); + } + + @Override + public PaneInfo pane() { + return newContext.pane(); + } + + @Override + public WindowingInternals windowingInternals() { + throw new UnsupportedOperationException(String.format( + "%s.%s.windowingInternals() is no longer supported. Please convert your %s to a %s", + OldDoFn.class.getSimpleName(), + OldDoFn.ProcessContext.class.getSimpleName(), + OldDoFn.class.getSimpleName(), + DoFn.class.getSimpleName())); + } + + @Override + public PipelineOptions getPipelineOptions() { + return newContext.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + newContext.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + newContext.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + newContext.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + newContext.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { + return newContext.createAggregator(name, combiner); + } + } + + private class AdaptedDoFn extends DoFn { + + @Setup + public void setup() throws Exception { + OldDoFn.this.setup(); + } + + @StartBundle + public void startBundle(Context c) throws Exception { + OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c)); + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + OldDoFn.this.processElement(OldDoFn.this.new AdaptedProcessContext(c)); + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c)); + } + + @Teardown + public void teardown() throws Exception { + OldDoFn.this.teardown(); + } + + @Override + public Duration getAllowedTimestampSkew() { + return OldDoFn.this.getAllowedTimestampSkew(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + OldDoFn.this.populateDisplayData(builder); + } + + @Override + public TypeDescriptor getInputTypeDescriptor() { + return OldDoFn.this.getInputTypeDescriptor(); + } + + @Override + Collection> getAggregators() { + return OldDoFn.this.getAggregators(); + } + + @Override + public TypeDescriptor getOutputTypeDescriptor() { + return OldDoFn.this.getOutputTypeDescriptor(); + } + } + + /** + * A {@link ProcessContext} for an {@link OldDoFn} that implements + * {@link RequiresWindowAccess}, via a context for a proper {@link DoFn}. + */ + private class AdaptedRequiresWindowAccessProcessContext extends AdaptedProcessContext { + + private final BoundedWindow window; + + public AdaptedRequiresWindowAccessProcessContext( + DoFn.ProcessContext newContext, + BoundedWindow window) { + super(newContext); + this.window = window; + } + + @Override + public BoundedWindow window() { + return window; + } + } + + private class AdaptedRequiresWindowAccessDoFn extends DoFn { + + @Setup + public void setup() throws Exception { + OldDoFn.this.setup(); + } + + @StartBundle + public void startBundle(Context c) throws Exception { + OldDoFn.this.startBundle(OldDoFn.this.new AdaptedContext(c)); + } + + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + OldDoFn.this.processElement( + OldDoFn.this.new AdaptedRequiresWindowAccessProcessContext(c, window)); + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + OldDoFn.this.finishBundle(OldDoFn.this.new AdaptedContext(c)); + } + + @Teardown + public void teardown() throws Exception { + OldDoFn.this.teardown(); + } + + @Override + public Duration getAllowedTimestampSkew() { + return OldDoFn.this.getAllowedTimestampSkew(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + OldDoFn.this.populateDisplayData(builder); + } + + @Override + public TypeDescriptor getInputTypeDescriptor() { + return OldDoFn.this.getInputTypeDescriptor(); + } + + @Override + public TypeDescriptor getOutputTypeDescriptor() { + return OldDoFn.this.getOutputTypeDescriptor(); + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index b141d51150ba..50a7082cdad5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -18,7 +18,13 @@ package org.apache.beam.sdk.transforms.reflect; import java.io.Serializable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.util.UserCodeException; /** Static utilities for working with {@link DoFnInvoker}. */ public class DoFnInvokers { @@ -36,22 +42,137 @@ public static DoFnInvoker invokerFor( return ByteBuddyDoFnInvokerFactory.only().newByteBuddyInvoker(fn); } + private DoFnInvokers() {} + /** - * Temporarily retained for compatibility with Dataflow worker. - * TODO: delete this when Dataflow worker is fixed to call {@link #invokerFor(DoFn)}. + * Returns a {@link DoFnInvoker} for the given {@link Object}, which should be either a {@link + * DoFn} or an {@link OldDoFn}. The expected use would be to deserialize a user's function as an + * {@link Object} and then pass it to this method, so there is no need to statically specify what + * sort of object it is. * - * @deprecated Use {@link #invokerFor(DoFn)}. + * @deprecated this is to be used only as a migration path for decoupling upgrades */ - @SuppressWarnings("unchecked") @Deprecated - public static DoFnInvoker invokerFor( - Serializable deserializedFn) { + public static DoFnInvoker invokerFor(Serializable deserializedFn) { if (deserializedFn instanceof DoFn) { - return invokerFor((DoFn) deserializedFn); + return invokerFor((DoFn) deserializedFn); + } else if (deserializedFn instanceof OldDoFn) { + return new OldDoFnInvoker<>((OldDoFn) deserializedFn); + } else { + throw new IllegalArgumentException( + String.format( + "Cannot create a %s for %s; it should be either a %s or an %s.", + DoFnInvoker.class.getSimpleName(), + deserializedFn.toString(), + DoFn.class.getSimpleName(), + OldDoFn.class.getSimpleName())); } - throw new UnsupportedOperationException( - "Only DoFn supported, was: " + deserializedFn.getClass()); } - private DoFnInvokers() {} + /** @deprecated use {@link DoFnInvokers#invokerFor(DoFn)}. */ + @Deprecated public static final DoFnInvokers INSTANCE = new DoFnInvokers(); + + /** @deprecated use {@link DoFnInvokers#invokerFor(DoFn)}. */ + @Deprecated + public DoFnInvoker invokerFor(Object deserializedFn) { + return (DoFnInvoker) DoFnInvokers.invokerFor((Serializable) deserializedFn); + } + + + static class OldDoFnInvoker implements DoFnInvoker { + + private final OldDoFn fn; + + public OldDoFnInvoker(OldDoFn fn) { + this.fn = fn; + } + + @Override + public DoFn.ProcessContinuation invokeProcessElement( + ArgumentProvider extra) { + // The outer DoFn is immaterial - it exists only to avoid typing InputT and OutputT repeatedly + DoFn.ProcessContext newCtx = + extra.processContext(new DoFn() {}); + OldDoFn.ProcessContext oldCtx = + DoFnAdapters.adaptProcessContext(fn, newCtx, extra); + try { + fn.processElement(oldCtx); + return DoFn.ProcessContinuation.stop(); + } catch (Throwable exc) { + throw UserCodeException.wrap(exc); + } + } + + @Override + public void invokeOnTimer(String timerId, ArgumentProvider arguments) { + throw new UnsupportedOperationException( + String.format("Timers are not supported for %s", OldDoFn.class.getSimpleName())); + } + + @Override + public void invokeStartBundle(DoFn.Context c) { + OldDoFn.Context oldCtx = DoFnAdapters.adaptContext(fn, c); + try { + fn.startBundle(oldCtx); + } catch (Throwable exc) { + throw UserCodeException.wrap(exc); + } + } + + @Override + public void invokeFinishBundle(DoFn.Context c) { + OldDoFn.Context oldCtx = DoFnAdapters.adaptContext(fn, c); + try { + fn.finishBundle(oldCtx); + } catch (Throwable exc) { + throw UserCodeException.wrap(exc); + } + } + + @Override + public void invokeSetup() { + try { + fn.setup(); + } catch (Throwable exc) { + throw UserCodeException.wrap(exc); + } + } + + @Override + public void invokeTeardown() { + try { + fn.teardown(); + } catch (Throwable exc) { + throw UserCodeException.wrap(exc); + } + } + + @Override + public RestrictionT invokeGetInitialRestriction(InputT element) { + throw new UnsupportedOperationException("OldDoFn is not splittable"); + } + + @Override + public Coder invokeGetRestrictionCoder( + CoderRegistry coderRegistry) { + throw new UnsupportedOperationException("OldDoFn is not splittable"); + } + + @Override + public void invokeSplitRestriction( + InputT element, RestrictionT restriction, DoFn.OutputReceiver receiver) { + throw new UnsupportedOperationException("OldDoFn is not splittable"); + } + + @Override + public > + TrackerT invokeNewTracker(RestrictionT restriction) { + throw new UnsupportedOperationException("OldDoFn is not splittable"); + } + + @Override + public DoFn getFn() { + throw new UnsupportedOperationException("getFn is not supported for OldDoFn"); + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 4c6bee13f79d..4233b390a216 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -25,6 +25,7 @@ import static org.junit.Assert.assertThat; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -731,4 +732,39 @@ public void onMyTimer(IntervalWindow w) { invoker.invokeOnTimer(timerId, mockArgumentProvider); assertThat(fn.window, equalTo(testWindow)); } + + private class OldDoFnIdentity extends OldDoFn { + public void processElement(ProcessContext c) {} + } + + @Test + public void testOldDoFnProcessElement() throws Exception { + new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn) + .invokeProcessElement(mockArgumentProvider); + verify(mockOldDoFn).processElement(any(OldDoFn.ProcessContext.class)); + } + + @Test + public void testOldDoFnStartBundle() throws Exception { + new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeStartBundle(mockProcessContext); + verify(mockOldDoFn).startBundle(any(OldDoFn.Context.class)); + } + + @Test + public void testOldDoFnFinishBundle() throws Exception { + new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeFinishBundle(mockProcessContext); + verify(mockOldDoFn).finishBundle(any(OldDoFn.Context.class)); + } + + @Test + public void testOldDoFnSetup() throws Exception { + new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeSetup(); + verify(mockOldDoFn).setup(); + } + + @Test + public void testOldDoFnTeardown() throws Exception { + new DoFnInvokers.OldDoFnInvoker<>(mockOldDoFn).invokeTeardown(); + verify(mockOldDoFn).teardown(); + } } From 4aa0ee1436a8d94f7c1c75bd0151790d14635c64 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 16 Dec 2016 15:26:32 -0800 Subject: [PATCH 217/279] Revert "Removes ArgumentProvider.windowingInternals" This reverts commit f3e8a0383bf9cb3f9452e0364f7deba113cadff9. --- .../beam/runners/core/DoFnAdapters.java | 14 +++++ .../beam/runners/core/SimpleDoFnRunner.java | 57 +++++++++++++++++++ .../beam/runners/core/SplittableParDo.java | 7 +++ .../beam/sdk/transforms/DoFnTester.java | 7 +++ .../sdk/transforms/reflect/DoFnInvoker.java | 20 +++++++ .../transforms/reflect/DoFnInvokersTest.java | 6 ++ 6 files changed, 111 insertions(+) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java index a4002daf9028..fc5847c75a7d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Timer; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; @@ -354,6 +355,14 @@ public OnTimerContext onTimerContext(DoFn doFn) { "Timers are not supported for OldDoFn"); } + @Override + public WindowingInternals windowingInternals() { + // The OldDoFn doesn't allow us to ask for these outside ProcessElements, so this + // should be unreachable. + throw new UnsupportedOperationException( + "Can only get WindowingInternals in processElement"); + } + @Override public DoFn.InputProvider inputProvider() { throw new UnsupportedOperationException("inputProvider() exists only for testing"); @@ -466,6 +475,11 @@ public OnTimerContext onTimerContext(DoFn doFn) { throw new UnsupportedOperationException("Timers are not supported for OldDoFn"); } + @Override + public WindowingInternals windowingInternals() { + return context.windowingInternals(); + } + @Override public DoFn.InputProvider inputProvider() { throw new UnsupportedOperationException("inputProvider() exists only for testing"); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index d504b403002c..29ef3ef73935 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -51,10 +51,13 @@ import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.Timer; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; 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.State; +import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateSpec; @@ -415,6 +418,11 @@ public OutputReceiver outputReceiver() { throw new UnsupportedOperationException("OutputReceiver is for testing only."); } + @Override + public WindowingInternals windowingInternals() { + throw new UnsupportedOperationException("WindowingInternals are unsupported."); + } + @Override public RestrictionTracker restrictionTracker() { throw new UnsupportedOperationException( @@ -625,5 +633,54 @@ public Timer timer(String timerId) { throw new UnsupportedOperationException("Timer parameters are not supported."); } + @Override + public WindowingInternals windowingInternals() { + return new WindowingInternals() { + @Override + public Collection windows() { + return windowedValue.getWindows(); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public TimerInternals timerInternals() { + return context.stepContext.timerInternals(); + } + + @Override + public StateInternals stateInternals() { + return stepContext.stateInternals(); + } + + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + throw new UnsupportedOperationException("A DoFn cannot output to a different window"); + } + + @Override + public void sideOutputWindowedValue( + TupleTag tag, + SideOutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + throw new UnsupportedOperationException( + "A DoFn cannot side output to a different window"); + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { + return context.sideInput(view, sideInputWindow); + } + }; + } } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index f8d12ecb81f1..e6a2466b2f17 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -53,6 +53,7 @@ import org.apache.beam.sdk.util.Timer; 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.State; import org.apache.beam.sdk.util.state.StateInternals; @@ -683,6 +684,12 @@ public DoFn.OutputReceiver outputReceiver() { throw new IllegalStateException("Unexpected extra context access on a splittable DoFn"); } + @Override + public WindowingInternals windowingInternals() { + // DoFnSignatures should have verified that this DoFn doesn't access extra context. + throw new IllegalStateException("Unexpected extra context access on a splittable DoFn"); + } + @Override public TrackerT restrictionTracker() { return tracker; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index b2c3fd5ed2e9..2d8684aaaff7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -48,6 +48,7 @@ import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; @@ -326,6 +327,12 @@ public DoFn.OutputReceiver outputReceiver() { "Not expected to access OutputReceiver from DoFnTester"); } + @Override + public WindowingInternals windowingInternals() { + throw new UnsupportedOperationException( + "Not expected to access WindowingInternals from a new DoFn"); + } + @Override public RestrictionTracker restrictionTracker() { throw new UnsupportedOperationException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index 354578e2a831..97ac9d3d53c7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -27,9 +27,11 @@ import org.apache.beam.sdk.transforms.DoFn.StartBundle; import org.apache.beam.sdk.transforms.DoFn.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Timer; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.State; /** @@ -119,6 +121,19 @@ interface ArgumentProvider { /** A placeholder for testing purposes. */ OutputReceiver outputReceiver(); + /** + * For migration from {@link OldDoFn} to {@link DoFn}, provide a {@link WindowingInternals} so + * an {@link OldDoFn} can be run via {@link DoFnInvoker}. + * + *

      This is not exposed via the reflective capabilities of {@link DoFn}. + * + * @deprecated Please port occurences of {@link OldDoFn} to {@link DoFn}. If they require state + * and timers, they will need to wait for the arrival of those features. Do not introduce + * new uses of this method. + */ + @Deprecated + WindowingInternals windowingInternals(); + /** * If this is a splittable {@link DoFn}, returns the {@link RestrictionTracker} associated with * the current {@link ProcessElement} call. @@ -164,6 +179,11 @@ public OutputReceiver outputReceiver() { return null; } + @Override + public WindowingInternals windowingInternals() { + return null; + } + @Override public State state(String stateId) { return null; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 4233b390a216..456a6ebff0a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker.FakeArgumentProvider; import org.apache.beam.sdk.transforms.reflect.testhelper.DoFnInvokersTestHelper; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; @@ -51,6 +52,7 @@ import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; @@ -76,14 +78,18 @@ public class DoFnInvokersTest { @Mock private IntervalWindow mockWindow; @Mock private DoFn.InputProvider mockInputProvider; @Mock private DoFn.OutputReceiver mockOutputReceiver; + @Mock private WindowingInternals mockWindowingInternals; @Mock private DoFnInvoker.ArgumentProvider mockArgumentProvider; + @Mock private OldDoFn mockOldDoFn; + @Before public void setUp() { MockitoAnnotations.initMocks(this); when(mockArgumentProvider.window()).thenReturn(mockWindow); when(mockArgumentProvider.inputProvider()).thenReturn(mockInputProvider); when(mockArgumentProvider.outputReceiver()).thenReturn(mockOutputReceiver); + when(mockArgumentProvider.windowingInternals()).thenReturn(mockWindowingInternals); when(mockArgumentProvider.processContext(Matchers.any())).thenReturn(mockProcessContext); } From 954e57d7696fd14f7d1015f4e40f025ef8538802 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 16 Dec 2016 15:37:02 -0800 Subject: [PATCH 218/279] Revert "Moves DoFnAdapters to runners-core" This reverts commit 33ed3238e2b3899cff061be3056c5cc29fc60a04. --- .../apex/translation/WindowBoundTranslator.java | 2 +- .../operators/ApexGroupByKeyOperator.java | 2 +- .../translation/operators/ApexParDoOperator.java | 2 +- .../beam/runners/core/SimpleOldDoFnRunner.java | 4 ++-- .../runners/core/GroupAlsoByWindowsProperties.java | 2 +- .../translation/functions/FlinkDoFnFunction.java | 2 +- .../functions/FlinkMultiOutputDoFnFunction.java | 2 +- .../functions/FlinkProcessContextBase.java | 2 +- .../wrappers/streaming/DoFnOperator.java | 2 +- .../beam/sdk/transforms/AggregatorRetriever.java | 13 ++----------- .../apache/beam/sdk/transforms}/DoFnAdapters.java | 8 ++------ .../org/apache/beam/sdk/transforms/OldDoFn.java | 2 +- .../org/apache/beam/sdk/transforms/NoOpOldDoFn.java | 2 +- 13 files changed, 16 insertions(+), 29 deletions(-) rename {runners/core-java/src/main/java/org/apache/beam/runners/core => sdks/java/core/src/main/java/org/apache/beam/sdk/transforms}/DoFnAdapters.java (98%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java index ef049e19ce88..33b9269f9c43 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/WindowBoundTranslator.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; import org.apache.beam.runners.core.AssignWindowsDoFn; -import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index 4af7ff0330e2..48ac177ead7b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -413,7 +413,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - public Aggregator createAggregatorInternal( + protected Aggregator createAggregatorInternal( String name, Combine.CombineFn combiner) { throw new UnsupportedOperationException(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java index 4538fb5a052b..a3d3a97e2ce0 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java @@ -38,7 +38,6 @@ import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; import org.apache.beam.runners.apex.translation.utils.ValueAndCoderKryoSerializable; import org.apache.beam.runners.core.AggregatorFactory; -import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; @@ -49,6 +48,7 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 7d9320062f27..1048fdcc09ad 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -322,7 +322,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - public Aggregator createAggregatorInternal( + protected Aggregator createAggregatorInternal( String name, CombineFn combiner) { checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null"); return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner); @@ -504,7 +504,7 @@ public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { } @Override - public Aggregator + protected Aggregator createAggregatorInternal( String name, CombineFn combiner) { return context.createAggregatorInternal(name, combiner); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java index ef01106fdf77..97b67c653c9c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java @@ -744,7 +744,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - public Aggregator createAggregatorInternal( + protected Aggregator createAggregatorInternal( String name, CombineFn combiner) { throw new UnsupportedOperationException(); } 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 2a4a68e7b5bb..ed200d58aa57 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,10 +18,10 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.WindowedValue; 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 a97bd46fc77b..7f6a4369ad85 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,10 +18,10 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java index 53b98038adb9..6afca38dcce4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContextBase.java @@ -252,7 +252,7 @@ protected abstract void outputWithTimestampAndWindow( public abstract void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp); @Override - public Aggregator + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { @SuppressWarnings("unchecked") SerializableFnAggregatorWrapper result = diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 001e3b68ce0c..870430844a7c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.core.AggregatorFactory; -import org.apache.beam.runners.core.DoFnAdapters; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; @@ -42,6 +41,7 @@ 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.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java index b1d3ead72ee2..ce47e22fda06 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AggregatorRetriever.java @@ -18,10 +18,9 @@ package org.apache.beam.sdk.transforms; import java.util.Collection; -import java.util.Map; /** - * An internal class for extracting {@link Aggregator Aggregators} from {@link DoFn DoFns}. + * An internal class for extracting {@link Aggregator Aggregators} from {@link OldDoFn DoFns}. */ public final class AggregatorRetriever { private AggregatorRetriever() { @@ -29,17 +28,9 @@ private AggregatorRetriever() { } /** - * Returns the {@link Aggregator Aggregators} created by the provided {@link DoFn}. + * Returns the {@link Aggregator Aggregators} created by the provided {@link OldDoFn}. */ public static Collection> getAggregators(DoFn fn) { return fn.getAggregators(); } - - /** - * Returns the {@link DelegatingAggregator delegating aggregators} created by the provided {@link - * DoFn}. - */ - public static Map> getDelegatingAggregators(DoFn fn) { - return fn.aggregators; - } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index fc5847c75a7d..e15b08b46d79 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -15,20 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core; +package org.apache.beam.sdk.transforms; import java.io.IOException; import java.util.Collection; import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.AggregatorRetriever; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; @@ -209,7 +205,7 @@ private static class SimpleDoFnAdapter extends OldDoFn invoker; SimpleDoFnAdapter(DoFn fn) { - super(AggregatorRetriever.getDelegatingAggregators(fn)); + super(fn.aggregators); this.fn = fn; this.invoker = DoFnInvokers.invokerFor(fn); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java index d1bb42bec7f6..2d2c1fde737b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/OldDoFn.java @@ -208,7 +208,7 @@ public abstract void sideOutputWithTimestamp( * context */ @Experimental(Kind.AGGREGATOR) - public abstract Aggregator + protected abstract Aggregator createAggregatorInternal(String name, CombineFn combiner); /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java index 0db130db59dd..504480b66cea 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/NoOpOldDoFn.java @@ -63,7 +63,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { } @Override - public Aggregator + protected Aggregator createAggregatorInternal(String name, CombineFn combiner) { return null; } From 45ed5c70c18a806d0fc2e7385886285206fd18e4 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 16 Dec 2016 16:33:51 -0800 Subject: [PATCH 219/279] Revert "Move InMemoryTimerInternals to runners-core" This reverts commit ec0bf7b4023ff75f4ec6723d2e77ed507eb57c51. --- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 1 + .../beam/runners/core/ReduceFnTester.java | 1 + .../runners/core/SplittableParDoTest.java | 16 ++------- .../triggers/TriggerStateMachineTester.java | 2 +- .../translation/SparkGroupAlsoByWindowFn.java | 2 +- .../beam/sdk/transforms/DoFnTester.java | 36 +++++++++++++++++++ .../util/state}/InMemoryTimerInternals.java | 3 +- .../state}/InMemoryTimerInternalsTest.java | 4 +-- 8 files changed, 45 insertions(+), 20 deletions(-) rename {runners/core-java/src/main/java/org/apache/beam/runners/core => sdks/java/core/src/main/java/org/apache/beam/sdk/util/state}/InMemoryTimerInternals.java (99%) rename {runners/core-java/src/test/java/org/apache/beam/runners/core => sdks/java/core/src/test/java/org/apache/beam/sdk/util/state}/InMemoryTimerInternalsTest.java (97%) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index efcd771d01b9..918919170ec6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.joda.time.Instant; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 890195a0d3c1..db0cf9186a7d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -73,6 +73,7 @@ import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; +import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index 41d419baa8b2..cf96b660bea6 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -28,7 +28,6 @@ import static org.junit.Assert.assertTrue; import java.io.Serializable; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -191,8 +190,6 @@ private static class ProcessFnTester< tester; private Instant currentProcessingTime; - private InMemoryTimerInternals timerInternals; - ProcessFnTester( Instant currentProcessingTime, DoFn fn, @@ -203,7 +200,6 @@ private static class ProcessFnTester< new SplittableParDo.ProcessFn<>( fn, inputCoder, restrictionCoder, IntervalWindow.getCoder()); this.tester = DoFnTester.of(processFn); - this.timerInternals = new InMemoryTimerInternals(); processFn.setStateInternalsFactory( new StateInternalsFactory() { @Override @@ -215,7 +211,7 @@ public StateInternals stateInternalsForKey(String key) { new TimerInternalsFactory() { @Override public TimerInternals timerInternalsForKey(String key) { - return timerInternals; + return tester.getTimerInternals(); } }); processFn.setOutputWindowedValue( @@ -251,7 +247,7 @@ public void sideOutputWindowedValue( // through the state/timer/output callbacks. this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE); this.tester.startBundle(); - timerInternals.advanceProcessingTime(currentProcessingTime); + this.tester.advanceProcessingTime(currentProcessingTime); this.currentProcessingTime = currentProcessingTime; } @@ -289,13 +285,7 @@ void startElement( */ boolean advanceProcessingTimeBy(Duration duration) throws Exception { currentProcessingTime = currentProcessingTime.plus(duration); - timerInternals.advanceProcessingTime(currentProcessingTime); - - List timers = new ArrayList<>(); - TimerInternals.TimerData nextTimer; - while ((nextTimer = timerInternals.removeNextProcessingTimer()) != null) { - timers.add(nextTimer); - } + List timers = tester.advanceProcessingTime(currentProcessingTime); if (timers.isEmpty()) { return false; } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 2a626d4311d5..be63c0644e27 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -34,7 +34,6 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.ActiveWindowSet; import org.apache.beam.runners.core.ActiveWindowSet.MergeCallback; -import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.MergingActiveWindowSet; import org.apache.beam.runners.core.NonMergingActiveWindowSet; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -47,6 +46,7 @@ import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; +import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java index 5432d58994af..87d3f5059039 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java @@ -23,7 +23,6 @@ import java.util.Collection; import java.util.List; import org.apache.beam.runners.core.GroupAlsoByWindowsDoFn; -import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; import org.apache.beam.runners.core.SystemReduceFn; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 2d8684aaaff7..93b3f5954898 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -46,10 +46,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.Timer; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.InMemoryStateInternals; +import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; @@ -141,6 +143,10 @@ public StateInternals getStateInternals() { return (StateInternals) stateInternals; } + public TimerInternals getTimerInternals() { + return timerInternals; + } + /** * When a {@link DoFnTester} should clone the {@link DoFn} under test and how it should manage * the lifecycle of the {@link DoFn}. @@ -227,6 +233,7 @@ public void startBundle() throws Exception { context.setupDelegateAggregators(); // State and timer internals are per-bundle. stateInternals = InMemoryStateInternals.forKey(new Object()); + timerInternals = new InMemoryTimerInternals(); try { fnInvoker.invokeStartBundle(context); } catch (UserCodeException e) { @@ -535,6 +542,34 @@ public AggregateT getAggregatorValue(Aggregator agg) return extractAggregatorValue(agg.getName(), agg.getCombineFn()); } + public List advanceInputWatermark(Instant newWatermark) { + try { + timerInternals.advanceInputWatermark(newWatermark); + final List firedTimers = new ArrayList<>(); + TimerInternals.TimerData timer; + while ((timer = timerInternals.removeNextEventTimer()) != null) { + firedTimers.add(timer); + } + return firedTimers; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public List advanceProcessingTime(Instant newProcessingTime) { + try { + timerInternals.advanceProcessingTime(newProcessingTime); + final List firedTimers = new ArrayList<>(); + TimerInternals.TimerData timer; + while ((timer = timerInternals.removeNextProcessingTimer()) != null) { + firedTimers.add(timer); + } + return firedTimers; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + private AggregateT extractAggregatorValue( String name, CombineFn combiner) { @SuppressWarnings("unchecked") @@ -779,6 +814,7 @@ private enum State { private Map, List>> outputs; private InMemoryStateInternals stateInternals; + private InMemoryTimerInternals timerInternals; /** The state of processing of the {@link DoFn} under test. */ private State state = State.UNINITIALIZED; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java similarity index 99% rename from runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java index b22fcb3d2c0a..44b44f06ead6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core; +package org.apache.beam.sdk.util.state; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -29,7 +29,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.state.StateNamespace; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java similarity index 97% rename from runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java index 2caa8742dea1..4a2763ccc76e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core; +package org.apache.beam.sdk.util.state; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -25,8 +25,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; -import org.apache.beam.sdk.util.state.StateNamespace; -import org.apache.beam.sdk.util.state.StateNamespaceForTest; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; From 6a4a699796fcf8a294ee0886658e6597bede0207 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 16 Dec 2016 16:26:27 -0800 Subject: [PATCH 220/279] View.asMap: minor javadoc fixes --- .../src/main/java/org/apache/beam/sdk/transforms/View.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java index 126679d179fd..d18a0c6048e4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java @@ -189,9 +189,9 @@ public static AsIterable asIterable() { /** * Returns a {@link View.AsMap} transform that takes a - * {@link PCollection PCollection<KV<K V>>} as + * {@link PCollection PCollection<KV<K, V>>} as * input and produces a {@link PCollectionView} mapping each window to - * a {@link Map Map>K, V>}. It is required that each key of the input be + * a {@link Map Map<K, V>}. It is required that each key of the input be * associated with a single value, per window. If this is not the case, precede this * view with {@code Combine.perKey}, as in the example below, or alternatively * use {@link View#asMultimap()}. From c1e1017d6e4d75aee7f32cc3d08b9e2a7c21dbb2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:09:06 -0800 Subject: [PATCH 221/279] Make TimerSpec and StateSpec fields accessible --- .../apache/beam/sdk/transforms/reflect/DoFnSignatures.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index e3ba9663bcaa..d72cea40a416 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -1040,6 +1040,8 @@ private static ImmutableMap analyzeTimerDeclarations( ErrorReporter errors, Class fnClazz) { Map declarations = new HashMap<>(); for (Field field : declaredFieldsWithAnnotation(DoFn.TimerId.class, fnClazz, DoFn.class)) { + // TimerSpec fields may generally be private, but will be accessed via the signature + field.setAccessible(true); String id = field.getAnnotation(DoFn.TimerId.class).value(); validateTimerField(errors, declarations, id, field); declarations.put(id, DoFnSignature.TimerDeclaration.create(id, field)); @@ -1205,6 +1207,8 @@ private static Map analyzeStateDeclarati Map declarations = new HashMap<>(); for (Field field : declaredFieldsWithAnnotation(DoFn.StateId.class, fnClazz, DoFn.class)) { + // StateSpec fields may generally be private, but will be accessed via the signature + field.setAccessible(true); String id = field.getAnnotation(DoFn.StateId.class).value(); if (declarations.containsKey(id)) { From 8af13b0102cda6c68601efa4119723900d12ca5c Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 23 Nov 2016 14:21:40 -0800 Subject: [PATCH 222/279] Add timer support to DoFnRunner(s) --- .../apache/beam/runners/core/DoFnRunner.java | 9 + .../core/LateDataDroppingDoFnRunner.java | 7 + .../core/PushbackSideInputDoFnRunner.java | 8 + .../beam/runners/core/SimpleDoFnRunner.java | 236 ++++++++++++++++- .../runners/core/SimpleOldDoFnRunner.java | 8 + .../core/PushbackSideInputDoFnRunnerTest.java | 41 +++ .../runners/core/SimpleDoFnRunnerTest.java | 247 ++++++++++++++++++ 7 files changed, 555 insertions(+), 1 deletion(-) create mode 100644 runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 501667ef0122..7c73a3491556 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -20,8 +20,11 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.joda.time.Instant; /** * An wrapper interface that represents the execution of a {@link DoFn}. @@ -38,6 +41,12 @@ public interface DoFnRunner { */ void processElement(WindowedValue elem); + /** + * Calls a {@link DoFn DoFn's} {@link DoFn.OnTimer @OnTimer} method for the given timer + * in the given window. + */ + void onTimer(String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain); + /** * Calls a {@link DoFn DoFn's} {@link DoFn.FinishBundle @FinishBundle} method and performs * additional tasks, such as flushing in-memory states. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 9bfe9aeed559..290171ad2280 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; @@ -72,6 +73,12 @@ public void processElement(WindowedValue> elem) { doFnRunner.processElement(elem.withValue(keyedWorkItem)); } + @Override + public void onTimer(String timerId, BoundedWindow window, Instant timestamp, + TimeDomain timeDomain) { + doFnRunner.onTimer(timerId, window, timestamp, timeDomain); + } + @Override public void finishBundle() { doFnRunner.finishBundle(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index 0bb9153a6bed..2962832a5096 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -25,8 +25,10 @@ import java.util.Set; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.joda.time.Instant; /** * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning @@ -109,6 +111,12 @@ public void processElement(WindowedValue elem) { underlying.processElement(elem); } + @Override + public void onTimer(String timerId, BoundedWindow window, Instant timestamp, + TimeDomain timeDomain) { + underlying.onTimer(timerId, window, timestamp, timeDomain); + } + /** * Call the underlying {@link DoFnRunner#finishBundle()}. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 29ef3ef73935..a7d82bf52ed8 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -50,8 +50,10 @@ import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingInternals; @@ -64,6 +66,7 @@ import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; @@ -161,6 +164,35 @@ public void processElement(WindowedValue compressedElem) { } } + @Override + public void onTimer( + String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + + // The effective timestamp is when derived elements will have their timestamp set, if not + // otherwise specified. If this is an event time timer, then they have the timestamp of the + // timer itself. Otherwise, they are set to the input timestamp, which is by definition + // non-late. + Instant effectiveTimestamp; + switch (timeDomain) { + case EVENT_TIME: + effectiveTimestamp = timestamp; + break; + + case PROCESSING_TIME: + case SYNCHRONIZED_PROCESSING_TIME: + effectiveTimestamp = context.stepContext.timerInternals().currentInputWatermarkTime(); + break; + + default: + throw new IllegalArgumentException( + String.format("Unknown time domain: %s", timeDomain)); + } + + OnTimerArgumentProvider argumentProvider = + new OnTimerArgumentProvider<>(fn, context, window, effectiveTimestamp, timeDomain); + invoker.invokeOnTimer(timerId, argumentProvider); + } + private void invokeProcessElement(WindowedValue elem) { final DoFnProcessContext processContext = createProcessContext(elem); @@ -630,7 +662,13 @@ public State state(String stateId) { @Override public Timer timer(String timerId) { - throw new UnsupportedOperationException("Timer parameters are not supported."); + try { + TimerSpec spec = + (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn); + return new TimerInternalsTimer(getNamespace(), timerId, spec, stepContext.timerInternals()); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } } @Override @@ -682,5 +720,201 @@ public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { } }; } + + } + + /** + * A concrete implementation of {@link DoFnInvoker.ArgumentProvider} used for running a {@link + * DoFn} on a timer. + * + * @param the type of the {@link DoFn} (main) input elements + * @param the type of the {@link DoFn} (main) output elements + */ + private class OnTimerArgumentProvider + extends DoFn.OnTimerContext + implements DoFnInvoker.ArgumentProvider { + + final DoFn fn; + final DoFnContext context; + private final BoundedWindow window; + private final Instant timestamp; + private final TimeDomain timeDomain; + + /** Lazily initialized; should only be accessed via {@link #getNamespace()}. */ + private StateNamespace namespace; + + /** + * The state namespace for this context. + * + *

      Any call to {@link #getNamespace()} when more than one window is present will crash; this + * represents a bug in the runner or the {@link DoFnSignature}, since values must be in exactly + * one window when state or timers are relevant. + */ + private StateNamespace getNamespace() { + if (namespace == null) { + namespace = StateNamespaces.window(windowCoder, window); + } + return namespace; + } + + private OnTimerArgumentProvider( + DoFn fn, + DoFnContext context, + BoundedWindow window, + Instant timestamp, + TimeDomain timeDomain) { + fn.super(); + this.fn = fn; + this.context = context; + this.window = window; + this.timestamp = timestamp; + this.timeDomain = timeDomain; + } + + @Override + public Instant timestamp() { + return timestamp; + } + + @Override + public BoundedWindow window() { + return window; + } + + @Override + public TimeDomain timeDomain() { + return timeDomain; + } + + @Override + public Context context(DoFn doFn) { + throw new UnsupportedOperationException("Context parameters are not supported."); + } + + @Override + public ProcessContext processContext(DoFn doFn) { + throw new UnsupportedOperationException("ProcessContext parameters are not supported."); + } + + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + return this; + } + + @Override + public InputProvider inputProvider() { + throw new UnsupportedOperationException("InputProvider parameters are not supported."); + } + + @Override + public OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("OutputReceiver parameters are not supported."); + } + + @Override + public RestrictionTracker restrictionTracker() { + throw new UnsupportedOperationException("RestrictionTracker parameters are not supported."); + } + + @Override + public State state(String stateId) { + try { + StateSpec spec = + (StateSpec) signature.stateDeclarations().get(stateId).field().get(fn); + return stepContext + .stateInternals() + .state(getNamespace(), StateTags.tagForSpec(stateId, (StateSpec) spec)); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + @Override + public Timer timer(String timerId) { + try { + TimerSpec spec = + (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn); + return new TimerInternalsTimer(getNamespace(), timerId, spec, stepContext.timerInternals()); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + protected Aggregator createAggregator( + String name, + CombineFn combiner) { + throw new UnsupportedOperationException("Cannot createAggregator in @OnTimer method"); + } + + @Override + public WindowingInternals windowingInternals() { + throw new UnsupportedOperationException("WindowingInternals are unsupported."); + } + } + + private static class TimerInternalsTimer implements Timer { + private final TimerInternals timerInternals; + private final String timerId; + private final TimerSpec spec; + private final StateNamespace namespace; + + public TimerInternalsTimer( + StateNamespace namespace, String timerId, TimerSpec spec, TimerInternals timerInternals) { + this.namespace = namespace; + this.timerId = timerId; + this.spec = spec; + this.timerInternals = timerInternals; + } + + @Override + public void setForNowPlus(Duration durationFromNow) { + timerInternals.setTimer( + namespace, timerId, getCurrentTime().plus(durationFromNow), spec.getTimeDomain()); + } + + @Override + public void cancel() { + timerInternals.deleteTimer(namespace, timerId); + } + + private Instant getCurrentTime() { + switch(spec.getTimeDomain()) { + case EVENT_TIME: + return timerInternals.currentInputWatermarkTime(); + case PROCESSING_TIME: + return timerInternals.currentProcessingTime(); + case SYNCHRONIZED_PROCESSING_TIME: + return timerInternals.currentSynchronizedProcessingTime(); + default: + throw new IllegalStateException( + String.format("Timer created for unknown time domain %s", spec.getTimeDomain())); + } + } } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 1048fdcc09ad..342a4a869458 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -39,6 +39,7 @@ import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; +import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; @@ -107,6 +108,13 @@ public void processElement(WindowedValue elem) { } } + @Override + public void onTimer(String timerId, BoundedWindow window, Instant timestamp, + TimeDomain timeDomain) { + throw new UnsupportedOperationException( + String.format("Timers are not supported by %s", OldDoFn.class.getSimpleName())); + } + private void invokeProcessElement(WindowedValue elem) { final OldDoFn.ProcessContext processContext = createProcessContext(elem); // This can contain user code. Wrap it in case it throws an exception. diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java index 176ab267621e..a1cdbf6dce04 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.core; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; @@ -37,7 +38,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.hamcrest.Matchers; @@ -215,8 +219,33 @@ public void processElementNoSideInputs() { assertThat(underlying.inputElems, containsInAnyOrder(multiWindow)); } + /** Tests that a call to onTimer gets delegated. */ + @Test + public void testOnTimerCalled() { + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of()); + + String timerId = "fooTimer"; + IntervalWindow window = new IntervalWindow(new Instant(4), new Instant(16)); + Instant timestamp = new Instant(72); + + // Mocking is not easily compatible with annotation analysis, so we manually record + // the method call. + runner.onTimer(timerId, window, new Instant(timestamp), TimeDomain.EVENT_TIME); + + assertThat( + underlying.firedTimers, + contains( + TimerData.of( + timerId, + StateNamespaces.window(IntervalWindow.getCoder(), window), + timestamp, + TimeDomain.EVENT_TIME))); + } + private static class TestDoFnRunner implements DoFnRunner { List> inputElems; + List firedTimers; private boolean started = false; private boolean finished = false; @@ -224,6 +253,7 @@ private static class TestDoFnRunner implements DoFnRunner(); + firedTimers = new ArrayList<>(); } @Override @@ -231,6 +261,17 @@ public void processElement(WindowedValue elem) { inputElems.add(elem); } + @Override + public void onTimer(String timerId, BoundedWindow window, Instant timestamp, + TimeDomain timeDomain) { + firedTimers.add( + TimerData.of( + timerId, + StateNamespaces.window(IntervalWindow.getCoder(), (IntervalWindow) window), + timestamp, + timeDomain)); + } + @Override public void finishBundle() { finished = true; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java new file mode 100644 index 000000000000..f068c190ac3b --- /dev/null +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -0,0 +1,247 @@ +/* + * 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.core; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +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.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.Timer; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.TimerSpec; +import org.apache.beam.sdk.util.TimerSpecs; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Tests for {@link SimpleDoFnRunner}. */ +@RunWith(JUnit4.class) +public class SimpleDoFnRunnerTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Mock StepContext mockStepContext; + + @Mock TimerInternals mockTimerInternals; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + when(mockStepContext.timerInternals()).thenReturn(mockTimerInternals); + } + + @Test + public void testProcessElementExceptionsWrappedAsUserCodeException() { + ThrowingDoFn fn = new ThrowingDoFn(); + DoFnRunner runner = + new SimpleDoFnRunner<>( + null, + fn, + null, + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WindowingStrategy.of(new GlobalWindows())); + + thrown.expect(UserCodeException.class); + thrown.expectCause(is(fn.exceptionToThrow)); + + runner.processElement(WindowedValue.valueInGlobalWindow("anyValue")); + } + + @Test + public void testOnTimerExceptionsWrappedAsUserCodeException() { + ThrowingDoFn fn = new ThrowingDoFn(); + DoFnRunner runner = + new SimpleDoFnRunner<>( + null, + fn, + null, + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WindowingStrategy.of(new GlobalWindows())); + + thrown.expect(UserCodeException.class); + thrown.expectCause(is(fn.exceptionToThrow)); + + runner.onTimer( + ThrowingDoFn.TIMER_ID, + GlobalWindow.INSTANCE, + new Instant(0), + TimeDomain.EVENT_TIME); + } + + /** + * Tests that a users call to set a timer gets properly dispatched to the timer internals. From + * there on, it is the duty of the runner & step context to set it in whatever way is right for + * that runner. + */ + @Test + public void testTimerSet() { + WindowFn windowFn = new GlobalWindows(); + DoFnWithTimers fn = new DoFnWithTimers(windowFn.windowCoder()); + DoFnRunner runner = + new SimpleDoFnRunner<>( + null, + fn, + null, + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WindowingStrategy.of(new GlobalWindows())); + + // Setting the timer needs the current time, as it is set relative + Instant currentTime = new Instant(42); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(currentTime); + + runner.processElement(WindowedValue.valueInGlobalWindow("anyValue")); + + verify(mockTimerInternals) + .setTimer( + StateNamespaces.window(new GlobalWindows().windowCoder(), GlobalWindow.INSTANCE), + DoFnWithTimers.TIMER_ID, + currentTime.plus(DoFnWithTimers.TIMER_OFFSET), + TimeDomain.EVENT_TIME); + } + + /** + * Tests that {@link SimpleDoFnRunner#onTimer} properly dispatches to the underlying + * {@link DoFn}. + */ + @Test + public void testOnTimerCalled() { + WindowFn windowFn = new GlobalWindows(); + DoFnWithTimers fn = new DoFnWithTimers(windowFn.windowCoder()); + DoFnRunner runner = + new SimpleDoFnRunner<>( + null, + fn, + null, + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WindowingStrategy.of(windowFn)); + + Instant currentTime = new Instant(42); + Duration offset = Duration.millis(37); + + // Mocking is not easily compatible with annotation analysis, so we manually record + // the method call. + runner.onTimer( + DoFnWithTimers.TIMER_ID, + GlobalWindow.INSTANCE, + currentTime.plus(offset), + TimeDomain.EVENT_TIME); + + assertThat( + fn.onTimerInvocations, + contains( + TimerData.of( + DoFnWithTimers.TIMER_ID, + StateNamespaces.window(windowFn.windowCoder(), GlobalWindow.INSTANCE), + currentTime.plus(offset), + TimeDomain.EVENT_TIME))); + } + + static class ThrowingDoFn extends DoFn { + final Exception exceptionToThrow = new UnsupportedOperationException("Expected exception"); + + static final String TIMER_ID = "throwingTimerId"; + + @TimerId(TIMER_ID) + private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + throw exceptionToThrow; + } + + @OnTimer(TIMER_ID) + public void onTimer(OnTimerContext context) throws Exception { + throw exceptionToThrow; + } + } + + private static class DoFnWithTimers extends DoFn { + static final String TIMER_ID = "testTimerId"; + + static final Duration TIMER_OFFSET = Duration.millis(100); + + private final Coder windowCoder; + + // Mutable + List onTimerInvocations; + + DoFnWithTimers(Coder windowCoder) { + this.windowCoder = windowCoder; + this.onTimerInvocations = new ArrayList<>(); + } + + @TimerId(TIMER_ID) + private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void process(ProcessContext context, @TimerId(TIMER_ID) Timer timer) { + timer.setForNowPlus(TIMER_OFFSET); + } + + @OnTimer(TIMER_ID) + public void onTimer(OnTimerContext context) { + onTimerInvocations.add( + TimerData.of( + DoFnWithTimers.TIMER_ID, + StateNamespaces.window(windowCoder, (W) context.window()), + context.timestamp(), + context.timeDomain())); + } + } +} From 3b4c7d103c07e73d30b2ad534a17b3059232dbda Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 16 Dec 2016 13:43:54 -0800 Subject: [PATCH 223/279] Test that SimpleDoFnRunner wraps exceptions in startBundle and finishBundle --- .../runners/core/SimpleDoFnRunnerTest.java | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index f068c190ac3b..837a162d2033 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -151,6 +151,49 @@ public void testTimerSet() { TimeDomain.EVENT_TIME); } + @Test + public void testStartBundleExceptionsWrappedAsUserCodeException() { + ThrowingDoFn fn = new ThrowingDoFn(); + DoFnRunner runner = + new SimpleDoFnRunner<>( + null, + fn, + null, + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WindowingStrategy.of(new GlobalWindows())); + + thrown.expect(UserCodeException.class); + thrown.expectCause(is(fn.exceptionToThrow)); + + runner.startBundle(); + } + + @Test + public void testFinishBundleExceptionsWrappedAsUserCodeException() { + ThrowingDoFn fn = new ThrowingDoFn(); + DoFnRunner runner = + new SimpleDoFnRunner<>( + null, + fn, + null, + null, + null, + Collections.>emptyList(), + mockStepContext, + null, + WindowingStrategy.of(new GlobalWindows())); + + thrown.expect(UserCodeException.class); + thrown.expectCause(is(fn.exceptionToThrow)); + + runner.finishBundle(); + } + + /** * Tests that {@link SimpleDoFnRunner#onTimer} properly dispatches to the underlying * {@link DoFn}. @@ -200,6 +243,16 @@ static class ThrowingDoFn extends DoFn { @TimerId(TIMER_ID) private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + @StartBundle + public void startBundle(Context c) throws Exception { + throw exceptionToThrow; + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + throw exceptionToThrow; + } + @ProcessElement public void processElement(ProcessContext c) throws Exception { throw exceptionToThrow; From b78aa669831154f82266eb12ab795442c02f8977 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 16 Dec 2016 20:57:06 -0800 Subject: [PATCH 224/279] Use empty SideInputReader, fixes NPE in SimpleDoFnRunnerTest --- .../beam/runners/core/SimpleDoFnRunnerTest.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index 837a162d2033..ec5d375117e9 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; +import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerInternals; @@ -77,7 +78,7 @@ public void testProcessElementExceptionsWrappedAsUserCodeException() { new SimpleDoFnRunner<>( null, fn, - null, + NullSideInputReader.empty(), null, null, Collections.>emptyList(), @@ -98,7 +99,7 @@ public void testOnTimerExceptionsWrappedAsUserCodeException() { new SimpleDoFnRunner<>( null, fn, - null, + NullSideInputReader.empty(), null, null, Collections.>emptyList(), @@ -129,7 +130,7 @@ public void testTimerSet() { new SimpleDoFnRunner<>( null, fn, - null, + NullSideInputReader.empty(), null, null, Collections.>emptyList(), @@ -158,7 +159,7 @@ public void testStartBundleExceptionsWrappedAsUserCodeException() { new SimpleDoFnRunner<>( null, fn, - null, + NullSideInputReader.empty(), null, null, Collections.>emptyList(), @@ -179,7 +180,7 @@ public void testFinishBundleExceptionsWrappedAsUserCodeException() { new SimpleDoFnRunner<>( null, fn, - null, + NullSideInputReader.empty(), null, null, Collections.>emptyList(), @@ -206,7 +207,7 @@ public void testOnTimerCalled() { new SimpleDoFnRunner<>( null, fn, - null, + NullSideInputReader.empty(), null, null, Collections.>emptyList(), From 5f41deda509acbbbc6280323e583bb3c1af2dad2 Mon Sep 17 00:00:00 2001 From: Sela Date: Wed, 14 Dec 2016 12:20:08 +0200 Subject: [PATCH 225/279] [BEAM-853] Force streaming execution on batch pipelines for testing. Expose the adapted source. Force streaming execution, if set in PipelineOptions. Added test. --- .../runners/spark/SparkPipelineOptions.java | 5 + .../beam/runners/spark/TestSparkRunner.java | 80 +++++++++++- .../runners/spark/ForceStreamingTest.java | 123 ++++++++++++++++++ .../io/BoundedReadFromUnboundedSource.java | 14 +- 4 files changed, 217 insertions(+), 5 deletions(-) create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java index a2cd8879c154..04c559e86aef 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineOptions.java @@ -100,4 +100,9 @@ public String create(PipelineOptions options) { @Default.Boolean(false) boolean getUsesProvidedSparkContext(); void setUsesProvidedSparkContext(boolean value); + + @Description("A special flag that forces streaming in tests.") + @Default.Boolean(false) + boolean isForceStreaming(); + void setForceStreaming(boolean forceStreaming); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 2c26d84d4150..798ca4779804 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -19,16 +19,26 @@ package org.apache.beam.runners.spark; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import org.apache.beam.runners.core.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult.State; +import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.ValueWithRecordId; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; + /** * The SparkRunner translate operations defined on a pipeline to a representation executable * by Spark, and then submitting the job to Spark to be executed. If we wanted to run a Beam @@ -53,9 +63,12 @@ public final class TestSparkRunner extends PipelineRunner { private SparkRunner delegate; + private boolean isForceStreaming; + private int expectedNumberOfAssertions = 0; private TestSparkRunner(SparkPipelineOptions options) { this.delegate = SparkRunner.fromOptions(options); + this.isForceStreaming = options.isForceStreaming(); } public static TestSparkRunner fromOptions(PipelineOptions options) { @@ -65,19 +78,78 @@ public static TestSparkRunner fromOptions(PipelineOptions options) { return new TestSparkRunner(sparkOptions); } + /** + * Overrides for the test runner. + */ + @SuppressWarnings("unchecked") @Override - public - OutputT apply(PTransform transform, InputT input) { - return delegate.apply(transform, input); - }; + public OutputT apply( + PTransform transform, InputT input) { + // if the pipeline forces execution as a streaming pipeline, + // and the source is an adapted unbounded source (as bounded), + // read it as unbounded source via UnboundedReadFromBoundedSource. + if (isForceStreaming && transform instanceof BoundedReadFromUnboundedSource) { + return (OutputT) delegate.apply(new AdaptedBoundedAsUnbounded( + (BoundedReadFromUnboundedSource) transform), input); + } else { + // no actual override, simply counts asserting transforms in the pipeline. + if (transform instanceof PAssert.OneSideInputAssert + || transform instanceof PAssert.GroupThenAssert + || transform instanceof PAssert.GroupThenAssertForSingleton) { + expectedNumberOfAssertions += 1; + } + + return delegate.apply(transform, input); + } + } @Override public SparkPipelineResult run(Pipeline pipeline) { TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class); SparkPipelineResult result = delegate.run(pipeline); result.waitUntilFinish(); + + // make sure the test pipeline finished successfully. + State resultState = result.getState(); + assertThat( + String.format("Test pipeline result state was %s instead of %s", resultState, State.DONE), + resultState, + is(State.DONE)); assertThat(result, testPipelineOptions.getOnCreateMatcher()); assertThat(result, testPipelineOptions.getOnSuccessMatcher()); + + // if the pipeline was executed in streaming mode, validate aggregators. + if (isForceStreaming) { + // validate assertion succeeded (at least once). + int success = result.getAggregatorValue(PAssert.SUCCESS_COUNTER, Integer.class); + assertThat( + String.format( + "Expected %d successful assertions, but found %d.", + expectedNumberOfAssertions, success), + success, + is(expectedNumberOfAssertions)); + // validate assertion didn't fail. + int failure = result.getAggregatorValue(PAssert.FAILURE_COUNTER, Integer.class); + assertThat("Failure aggregator should be zero.", failure, is(0)); + } return result; } + + private static class AdaptedBoundedAsUnbounded extends PTransform> { + private final BoundedReadFromUnboundedSource source; + + AdaptedBoundedAsUnbounded(BoundedReadFromUnboundedSource source) { + this.source = source; + } + + @SuppressWarnings("unchecked") + @Override + public PCollection expand(PBegin input) { + PTransform>> replacingTransform = + new UnboundedReadFromBoundedSource<>(source.getAdaptedSource()); + return (PCollection) input.apply(replacingTransform) + .apply("StripIds", ParDo.of(new ValueWithRecordId.StripIdsDoFn())); + } + } + } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java new file mode 100644 index 000000000000..eb17eea9112f --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.spark; + +import static org.hamcrest.MatcherAssert.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.transforms.PTransform; +import org.junit.Test; + + +/** + * Test that we can "force streaming" on pipelines with {@link BoundedReadFromUnboundedSource} + * inputs using the {@link TestSparkRunner}. + * + *

      The test validates that when a pipeline reads from a {@link BoundedReadFromUnboundedSource}, + * with {@link SparkPipelineOptions#setStreaming(boolean)} true + * and using the {@link TestSparkRunner}; the {@link Read.Bounded} transform + * is replaced by an {@link Read.Unbounded} transform. + * + *

      This test does not execute a pipeline. + */ +public class ForceStreamingTest { + + @Test + public void test() throws IOException { + SparkPipelineOptions options = PipelineOptionsFactory.create().as(SparkPipelineOptions.class); + options.setRunner(TestSparkRunner.class); + // force streaming. + options.setForceStreaming(true); + + Pipeline pipeline = Pipeline.create(options); + + // apply the BoundedReadFromUnboundedSource. + @SuppressWarnings("unchecked") + BoundedReadFromUnboundedSource boundedRead = + Read.from(new FakeUnboundedSource()).withMaxNumRecords(-1); + //noinspection unchecked + pipeline.apply(boundedRead); + + UnboundedReadDetector unboundedReadDetector = new UnboundedReadDetector(); + pipeline.traverseTopologically(unboundedReadDetector); + + // assert that the applied BoundedReadFromUnboundedSource + // is being treated as an unbounded read. + assertThat("Expected to have an unbounded read.", unboundedReadDetector.isUnbounded); + } + + /** + * Traverses the Pipeline to check if the input is indeed a {@link Read.Unbounded}. + */ + private class UnboundedReadDetector extends Pipeline.PipelineVisitor.Defaults { + private boolean isUnbounded = false; + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + Class transformClass = node.getTransform().getClass(); + if (transformClass == Read.Unbounded.class) { + isUnbounded = true; + } + } + + } + + /** + * A fake {@link UnboundedSource} to satisfy the compiler. + */ + private static class FakeUnboundedSource extends UnboundedSource { + + @Override + public List generateInitialSplits( + int desiredNumSplits, + PipelineOptions options) throws Exception { + return null; + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, + CheckpointMark checkpointMark) throws IOException { + return null; + } + + @Override + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() { } + + @Override + public Coder getDefaultOutputCoder() { + return null; + } + } + +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index f2ef358c5165..84e304433784 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -27,6 +27,7 @@ import java.util.NoSuchElementException; import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Distinct; @@ -50,6 +51,7 @@ public class BoundedReadFromUnboundedSource extends PTransform source; private final long maxNumRecords; private final Duration maxReadTime; + private final BoundedSource> adaptedSource; private static final FluentBackoff BACKOFF_FACTORY = FluentBackoff.DEFAULT .withInitialBackoff(Duration.millis(10)) @@ -81,12 +83,22 @@ public BoundedReadFromUnboundedSource withMaxReadTime(Duration maxReadTime) { this.source = source; this.maxNumRecords = maxNumRecords; this.maxReadTime = maxReadTime; + this.adaptedSource = new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime); + } + + /** + * Returns an adapted {@link BoundedSource} wrapping the underlying {@link UnboundedSource}, + * with the specified bounds on number of records and read time. + */ + @Experimental + public BoundedSource> getAdaptedSource() { + return adaptedSource; } @Override public PCollection expand(PBegin input) { PCollection> read = Pipeline.applyTransform(input, - Read.from(new UnboundedToBoundedSourceAdapter<>(source, maxNumRecords, maxReadTime))); + Read.from(getAdaptedSource())); if (source.requiresDeduping()) { read = read.apply(Distinct.withRepresentativeValueFn( new SerializableFunction, byte[]>() { From 5fb4f5de9515db717818f1e3ffd7ca3c6eba5614 Mon Sep 17 00:00:00 2001 From: Sam McVeety Date: Fri, 16 Dec 2016 18:10:28 -0800 Subject: [PATCH 226/279] BigQueryIO: fix streaming write, typo in API and improve testing --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 23 ++++-- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 72 ++++++++++++------- 2 files changed, 63 insertions(+), 32 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 28049ed27492..7bb1e510e994 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -370,7 +370,8 @@ public String apply(TableReference from) { } } - private static class TableSpecToTableRef + @VisibleForTesting + static class TableSpecToTableRef implements SerializableFunction { @Override public TableReference apply(String from) { @@ -807,6 +808,7 @@ public TableReference getTable() { /** * Returns the query to be read, or {@code null} if reading from a table instead. */ + @Nullable public String getQuery() { return query == null ? null : query.get(); } @@ -814,7 +816,8 @@ public String getQuery() { /** * Returns the query to be read, or {@code null} if reading from a table instead. */ - public ValueProvider getQueryProivder() { + @Nullable + public ValueProvider getQueryProvider() { return query; } @@ -2813,7 +2816,8 @@ private static class TableRowInfo { * a randomUUID is generated only once per bucket of data. The actual unique * id is created by concatenating this randomUUID with a sequential number. */ - private static class TagWithUniqueIdsAndTable + @VisibleForTesting + static class TagWithUniqueIdsAndTable extends DoFn, TableRowInfo>> { /** TableSpec to write to. */ private final ValueProvider tableSpec; @@ -2830,8 +2834,12 @@ private static class TagWithUniqueIdsAndTable checkArgument(table == null ^ tableRefFunction == null, "Exactly one of table or tableRefFunction should be set"); if (table != null) { - if (table.isAccessible() && table.get().getProjectId() == null) { - table.get().setProjectId(options.as(BigQueryOptions.class).getProject()); + if (table.isAccessible() && Strings.isNullOrEmpty(table.get().getProjectId())) { + TableReference tableRef = table.get() + .setProjectId(options.as(BigQueryOptions.class).getProject()); + table = NestedValueProvider.of( + StaticValueProvider.of(toJsonString(tableRef)), + new JsonTableRefToTableRef()); } this.tableSpec = NestedValueProvider.of(table, new TableRefToTableSpec()); } else { @@ -2870,6 +2878,11 @@ public void populateDisplayData(DisplayData.Builder builder) { } } + @VisibleForTesting + ValueProvider getTableSpec() { + return tableSpec; + } + private String tableSpecFromWindow(BigQueryOptions options, BoundedWindow window) { if (tableSpec != null) { return tableSpec.get(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index b78316f7b7e7..dc566d2c8634 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -26,6 +26,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -2242,43 +2243,60 @@ public interface RuntimeTestOptions extends PipelineOptions { } @Test - public void testRuntimeOptionsNotCalledInApplyInputTable() throws IOException { + public void testRuntimeOptionsNotCalledInApplyInputTable() { RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); - FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService()); + bqOptions.setTempLocation("gs://testbucket/testdir"); Pipeline pipeline = TestPipeline.create(options); - pipeline - .apply(BigQueryIO.Read - .from(options.getInputTable()).withoutValidation() - .withTestServices(fakeBqServices)) - .apply(BigQueryIO.Write - .to(options.getOutputTable()) - .withSchema(NestedValueProvider.of( - options.getOutputSchema(), new JsonSchemaToTableSchema())) - .withTestServices(fakeBqServices) - .withoutValidation()); + BigQueryIO.Read.Bound read = BigQueryIO.Read.from( + options.getInputTable()).withoutValidation(); + pipeline.apply(read); + // Test that this doesn't throw. + DisplayData.from(read); } @Test - public void testRuntimeOptionsNotCalledInApplyInputQuery() throws IOException { + public void testRuntimeOptionsNotCalledInApplyInputQuery() { RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); - FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() - .withJobService(new FakeJobService()); + bqOptions.setTempLocation("gs://testbucket/testdir"); Pipeline pipeline = TestPipeline.create(options); + BigQueryIO.Read.Bound read = BigQueryIO.Read.fromQuery( + options.getInputQuery()).withoutValidation(); + pipeline.apply(read); + // Test that this doesn't throw. + DisplayData.from(read); + } + + @Test + public void testRuntimeOptionsNotCalledInApplyOutput() { + RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); + BigQueryOptions bqOptions = options.as(BigQueryOptions.class); + bqOptions.setTempLocation("gs://testbucket/testdir"); + Pipeline pipeline = TestPipeline.create(options); + BigQueryIO.Write.Bound write = BigQueryIO.Write + .to(options.getOutputTable()) + .withSchema(NestedValueProvider.of( + options.getOutputSchema(), new JsonSchemaToTableSchema())) + .withoutValidation(); pipeline - .apply(BigQueryIO.Read - .fromQuery(options.getInputQuery()).withoutValidation() - .withTestServices(fakeBqServices)) - .apply(BigQueryIO.Write - .to(options.getOutputTable()) - .withSchema(NestedValueProvider.of( - options.getOutputSchema(), new JsonSchemaToTableSchema())) - .withTestServices(fakeBqServices) - .withoutValidation()); + .apply(Create.of()) + .apply(write); + // Test that this doesn't throw. + DisplayData.from(write); + } + + @Test + public void testTagWithUniqueIdsAndTableProjectNotNullWithNvp() { + BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class); + bqOptions.setProject("project"); + BigQueryIO.TagWithUniqueIdsAndTable tag = + new BigQueryIO.TagWithUniqueIdsAndTable( + bqOptions, NestedValueProvider.of( + StaticValueProvider.of("data_set.table_name"), + new BigQueryIO.TableSpecToTableRef()), null); + TableReference table = BigQueryIO.parseTableSpec(tag.getTableSpec().get()); + assertNotNull(table.getProjectId()); } private static void testNumFiles(File tempDir, int expectedNumFiles) { From 6b055d2debe879816808b4c1ee847e34cc1df5c0 Mon Sep 17 00:00:00 2001 From: Joshua Litt Date: Sat, 17 Dec 2016 11:12:12 -0800 Subject: [PATCH 227/279] Change counter name in TestDataflowRunner --- .../dataflow/testing/TestDataflowRunner.java | 29 +++++++++++++++---- .../testing/TestDataflowRunnerTest.java | 16 +++++++++- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java index 4b0fcf218a75..056444877734 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java @@ -61,7 +61,12 @@ */ public class TestDataflowRunner extends PipelineRunner { private static final String TENTATIVE_COUNTER = "tentative"; - private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark"; + // See https://issues.apache.org/jira/browse/BEAM-1170 + // we need to either fix the API or pipe the DRAINED signal through + @VisibleForTesting + static final String LEGACY_WATERMARK_METRIC_SUFFIX = "windmill-data-watermark"; + @VisibleForTesting + static final String WATERMARK_METRIC_SUFFIX = "DataWatermark"; private static final long MAX_WATERMARK_VALUE = -2L; private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class); @@ -247,6 +252,23 @@ Optional checkForPAssertSuccess(DataflowPipelineJob job, @Nullable JobM return Optional.absent(); } + /** + * Checks wether a metric is a streaming watermark. + * + * @return true if the metric is a watermark. + */ + boolean isWatermark(MetricUpdate metric) { + if (metric.getName() == null || metric.getName().getName() == null) { + return false; // no name -> shouldn't happen, not the watermark + } + if (metric.getScalar() == null) { + return false; // no scalar value -> not the watermark + } + String name = metric.getName().getName(); + return name.endsWith(LEGACY_WATERMARK_METRIC_SUFFIX) + || name.endsWith(WATERMARK_METRIC_SUFFIX); + } + /** * Check watermarks of the streaming job. At least one watermark metric must exist. * @@ -256,10 +278,7 @@ Optional checkForPAssertSuccess(DataflowPipelineJob job, @Nullable JobM boolean atMaxWatermark(DataflowPipelineJob job, JobMetrics metrics) { boolean hasMaxWatermark = false; for (MetricUpdate metric : metrics.getMetrics()) { - if (metric.getName() == null - || metric.getName().getName() == null - || !metric.getName().getName().endsWith(WATERMARK_METRIC_SUFFIX) - || metric.getScalar() == null) { + if (!isWatermark(metric)) { continue; } BigDecimal watermark = (BigDecimal) metric.getScalar(); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java index 366c6a13835c..da5630b487b2 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.testing; +import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.LEGACY_WATERMARK_METRIC_SUFFIX; +import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.WATERMARK_METRIC_SUFFIX; import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -95,7 +97,6 @@ public class TestDataflowRunnerTest { @Mock private MockLowLevelHttpRequest request; @Mock private GcsUtil mockGcsUtil; - private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark"; private static final BigDecimal DEFAULT_MAX_WATERMARK = new BigDecimal(-2); private TestDataflowPipelineOptions options; @@ -410,6 +411,19 @@ public void testCheckMaxWatermarkWithSingleWatermarkAtMax() throws IOException { assertTrue(runner.atMaxWatermark(job, metrics)); } + @Test + public void testCheckMaxWatermarkWithLegacyWatermarkAtMax() throws IOException { + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); + Pipeline p = TestPipeline.create(options); + p.apply(Create.of(1, 2, 3)); + + TestDataflowRunner runner = (TestDataflowRunner) p.getRunner(); + JobMetrics metrics = buildJobMetrics(generateMockStreamingMetrics( + ImmutableMap.of(LEGACY_WATERMARK_METRIC_SUFFIX, DEFAULT_MAX_WATERMARK))); + doReturn(State.RUNNING).when(job).getState(); + assertTrue(runner.atMaxWatermark(job, metrics)); + } + @Test public void testCheckMaxWatermarkWithSingleWatermarkNotAtMax() throws IOException { DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); From 443b25a4d11201fb88f40da437ec7aab4b3e273f Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 13 Dec 2016 19:27:41 +0200 Subject: [PATCH 228/279] [BEAM-85, BEAM-298] Make TestPipeline a JUnit Rule checking proper usage --- .../apache/beam/sdk/testing/TestPipeline.java | 207 +++++++++++++++--- .../beam/sdk/testing/TestPipelineTest.java | 183 ++++++++++++++-- 2 files changed, 344 insertions(+), 46 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 493d4cc324b1..49ac3afd0f8a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -23,12 +23,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.base.Strings; +import com.google.common.collect.FluentIterable; import com.google.common.collect.Iterators; import java.io.IOException; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; import java.util.Map.Entry; import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; @@ -39,34 +44,39 @@ import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.TestCredential; import org.junit.experimental.categories.Category; +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runners.model.Statement; /** - * A creator of test pipelines that can be used inside of tests that can be - * configured to run locally or against a remote pipeline runner. + * A creator of test pipelines that can be used inside of tests that can be configured to run + * locally or against a remote pipeline runner. * - *

      It is recommended to tag hand-selected tests for this purpose using the - * {@link RunnableOnService} {@link Category} annotation, as each test run against a pipeline runner - * will utilize resources of that pipeline runner. + *

      It is recommended to tag hand-selected tests for this purpose using the {@link + * RunnableOnService} {@link Category} annotation, as each test run against a pipeline runner will + * utilize resources of that pipeline runner. * *

      In order to run tests on a pipeline runner, the following conditions must be met: + * *

        - *
      • System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline - * options. For example: - *
        {@code [
        + * 
      • System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline + * options. For example: + *
        {@code [
          *     "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner",
          *     "--project=mygcpproject",
          *     "--stagingLocation=gs://mygcsbucket/path"
          *     ]}
        * Note that the set of pipeline options required is pipeline runner specific. - *
      • - *
      • Jars containing the SDK and test classes must be available on the classpath.
      • + *
      • Jars containing the SDK and test classes must be available on the classpath. *
      * *

      Use {@link PAssert} for tests, as it integrates with this test harness in both direct and * remote execution modes. For example: + * *

      {@code
        * Pipeline p = TestPipeline.create();
        * PCollection output = ...
      @@ -76,19 +86,136 @@
        * p.run();
        * }
      * - *

      For pipeline runners, it is required that they must throw an {@link AssertionError} - * containing the message from the {@link PAssert} that failed. + *

      For pipeline runners, it is required that they must throw an {@link AssertionError} containing + * the message from the {@link PAssert} that failed. */ -public class TestPipeline extends Pipeline { +public class TestPipeline extends Pipeline implements TestRule { + + private static class PipelineRunEnforcement { + + protected boolean enableAutoRunIfMissing; + protected final Pipeline pipeline; + private boolean runInvoked; + + private PipelineRunEnforcement(final Pipeline pipeline) { + this.pipeline = pipeline; + } + + private void enableAutoRunIfMissing(final boolean enable) { + enableAutoRunIfMissing = enable; + } + + protected void beforePipelineExecution() { + runInvoked = true; + } + + protected void afterTestCompletion() { + if (!runInvoked && enableAutoRunIfMissing) { + pipeline.run().waitUntilFinish(); + } + } + } + + private static class PipelineAbandonedNodeEnforcement extends PipelineRunEnforcement { + + private List runVisitedNodes; + + private final Predicate isPAssertNode = + new Predicate() { + + @Override + public boolean apply(final TransformHierarchy.Node node) { + return node.getTransform() instanceof PAssert.GroupThenAssert + || node.getTransform() instanceof PAssert.GroupThenAssertForSingleton + || node.getTransform() instanceof PAssert.OneSideInputAssert; + } + }; + + private static class NodeRecorder extends PipelineVisitor.Defaults { + + private final List visited = new LinkedList<>(); + + @Override + public void leaveCompositeTransform(final TransformHierarchy.Node node) { + visited.add(node); + } + + @Override + public void visitPrimitiveTransform(final TransformHierarchy.Node node) { + visited.add(node); + } + } + + private PipelineAbandonedNodeEnforcement(final TestPipeline pipeline) { + super(pipeline); + } + + private List recordPipelineNodes(final Pipeline pipeline) { + final NodeRecorder nodeRecorder = new NodeRecorder(); + pipeline.traverseTopologically(nodeRecorder); + return nodeRecorder.visited; + } + + private void verifyPipelineExecution() { + final List pipelineNodes = recordPipelineNodes(pipeline); + if (runVisitedNodes != null && !runVisitedNodes.equals(pipelineNodes)) { + final boolean hasDanglingPAssert = + FluentIterable.from(pipelineNodes) + .filter(Predicates.not(Predicates.in(runVisitedNodes))) + .anyMatch(isPAssertNode); + if (hasDanglingPAssert) { + throw new AbandonedNodeException("The pipeline contains abandoned PAssert(s)."); + } else { + throw new AbandonedNodeException("The pipeline contains abandoned PTransform(s)."); + } + } else if (runVisitedNodes == null && !enableAutoRunIfMissing) { + throw new PipelineRunMissingException("The pipeline has not been run."); + } + } + + @Override + protected void beforePipelineExecution() { + super.beforePipelineExecution(); + runVisitedNodes = recordPipelineNodes(pipeline); + } + + @Override + protected void afterTestCompletion() { + super.afterTestCompletion(); + verifyPipelineExecution(); + } + } + + /** + * An exception thrown in case an abandoned {@link org.apache.beam.sdk.transforms.PTransform} is + * detected, that is, a {@link org.apache.beam.sdk.transforms.PTransform} that has not been run. + */ + public static class AbandonedNodeException extends RuntimeException { + + AbandonedNodeException(final String msg) { + super(msg); + } + } + + /** An exception thrown in case a test finishes without invoking {@link Pipeline#run()}. */ + public static class PipelineRunMissingException extends RuntimeException { + + PipelineRunMissingException(final String msg) { + super(msg); + } + } + static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions"; static final String PROPERTY_USE_DEFAULT_DUMMY_RUNNER = "beamUseDummyRunner"; private static final ObjectMapper MAPPER = new ObjectMapper(); + private PipelineRunEnforcement enforcement = new PipelineAbandonedNodeEnforcement(this); + /** * Creates and returns a new test pipeline. * - *

      Use {@link PAssert} to add tests, then call - * {@link Pipeline#run} to execute the pipeline and check the tests. + *

      Use {@link PAssert} to add tests, then call {@link Pipeline#run} to execute the pipeline and + * check the tests. */ public static TestPipeline create() { return fromOptions(testingPipelineOptions()); @@ -98,16 +225,30 @@ public static TestPipeline fromOptions(PipelineOptions options) { return new TestPipeline(PipelineRunner.fromOptions(options), options); } - private TestPipeline(PipelineRunner runner, PipelineOptions options) { + private TestPipeline( + final PipelineRunner runner, final PipelineOptions options) { super(runner, options); } + @Override + public Statement apply(final Statement statement, final Description description) { + return new Statement() { + + @Override + public void evaluate() throws Throwable { + statement.evaluate(); + enforcement.afterTestCompletion(); + } + }; + } + /** - * Runs this {@link TestPipeline}, unwrapping any {@code AssertionError} - * that is raised during testing. + * Runs this {@link TestPipeline}, unwrapping any {@code AssertionError} that is raised during + * testing. */ @Override public PipelineResult run() { + enforcement.beforePipelineExecution(); try { return super.run(); } catch (RuntimeException exc) { @@ -120,18 +261,28 @@ public PipelineResult run() { } } + public TestPipeline enableAbandonedNodeEnforcement(final boolean enable) { + enforcement = + enable ? new PipelineAbandonedNodeEnforcement(this) : new PipelineRunEnforcement(this); + + return this; + } + + public TestPipeline enableAutoRunIfMissing(final boolean enable) { + enforcement.enableAutoRunIfMissing(enable); + return this; + } + @Override public String toString() { return "TestPipeline#" + getOptions().as(ApplicationNameOptions.class).getAppName(); } - /** - * Creates {@link PipelineOptions} for testing. - */ + /** Creates {@link PipelineOptions} for testing. */ public static PipelineOptions testingPipelineOptions() { try { - @Nullable String beamTestPipelineOptions = - System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS); + @Nullable + String beamTestPipelineOptions = System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS); PipelineOptions options = Strings.isNullOrEmpty(beamTestPipelineOptions) @@ -155,13 +306,15 @@ public static PipelineOptions testingPipelineOptions() { IOChannelUtils.registerIOFactoriesAllowOverride(options); return options; } catch (IOException e) { - throw new RuntimeException("Unable to instantiate test options from system property " - + PROPERTY_BEAM_TEST_PIPELINE_OPTIONS + ":" - + System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), e); + throw new RuntimeException( + "Unable to instantiate test options from system property " + + PROPERTY_BEAM_TEST_PIPELINE_OPTIONS + + ":" + + System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), + e); } } - public static String[] convertToArgs(PipelineOptions options) { try { byte[] opts = MAPPER.writeValueAsBytes(options); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java index 03563f3d199d..d1797e76005c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java @@ -24,30 +24,54 @@ import static org.junit.Assert.assertThat; import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.Serializable; import java.util.Arrays; +import java.util.Collections; import java.util.Date; import java.util.List; import java.util.UUID; +import org.apache.beam.sdk.AggregatorRetrievalException; +import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PCollection; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; +import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.RuleChain; import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link TestPipeline}. */ @RunWith(JUnit4.class) -public class TestPipelineTest { - @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); - @Rule public ExpectedException thrown = ExpectedException.none(); +public class TestPipelineTest implements Serializable { + private static final List WORDS = Collections.singletonList("hi there"); + private static final String DUMMY = "expected"; + + private final transient TestPipeline pipeline = + TestPipeline.fromOptions(pipelineOptions()).enableAbandonedNodeEnforcement(true); + + private final transient ExpectedException exception = ExpectedException.none(); + + @Rule public transient TestRule restoreSystemProperties = new RestoreSystemProperties(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Rule public transient RuleChain ruleOrder = RuleChain.outerRule(exception).around(pipeline); @Test public void testCreationUsingDefaults() { @@ -57,13 +81,13 @@ public void testCreationUsingDefaults() { @Test public void testCreationOfPipelineOptions() throws Exception { ObjectMapper mapper = new ObjectMapper(); - String stringOptions = mapper.writeValueAsString(new String[]{ - "--runner=org.apache.beam.sdk.testing.CrashingRunner", - "--project=testProject" - }); + String stringOptions = + mapper.writeValueAsString( + new String[] { + "--runner=org.apache.beam.sdk.testing.CrashingRunner", "--project=testProject" + }); System.getProperties().put("beamTestPipelineOptions", stringOptions); - GcpOptions options = - TestPipeline.testingPipelineOptions().as(GcpOptions.class); + GcpOptions options = TestPipeline.testingPipelineOptions().as(GcpOptions.class); assertEquals(CrashingRunner.class, options.getRunner()); assertEquals(options.getProject(), "testProject"); } @@ -71,8 +95,10 @@ public void testCreationOfPipelineOptions() throws Exception { @Test public void testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase() throws Exception { PipelineOptions options = TestPipeline.testingPipelineOptions(); - assertThat(options.as(ApplicationNameOptions.class).getAppName(), startsWith( - "TestPipelineTest-testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase")); + assertThat( + options.as(ApplicationNameOptions.class).getAppName(), + startsWith( + "TestPipelineTest-testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase")); } @Test @@ -96,13 +122,13 @@ private TestPipeline nestedMethod() { @Test public void testConvertToArgs() { - String[] args = new String[]{"--tempLocation=Test_Location"}; + String[] args = new String[] {"--tempLocation=Test_Location"}; PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class); String[] arr = TestPipeline.convertToArgs(options); List lst = Arrays.asList(arr); assertEquals(lst.size(), 2); - assertThat(lst, containsInAnyOrder("--tempLocation=Test_Location", - "--appName=TestPipelineTest")); + assertThat( + lst, containsInAnyOrder("--tempLocation=Test_Location", "--appName=TestPipelineTest")); } @Test @@ -131,8 +157,8 @@ public void testMatcherSerializationDeserialization() { opts.setOnSuccessMatcher(m2); String[] arr = TestPipeline.convertToArgs(opts); - TestPipelineOptions newOpts = PipelineOptionsFactory.fromArgs(arr) - .as(TestPipelineOptions.class); + TestPipelineOptions newOpts = + PipelineOptionsFactory.fromArgs(arr).as(TestPipelineOptions.class); assertEquals(m1, newOpts.getOnCreateMatcher()); assertEquals(m2, newOpts.getOnSuccessMatcher()); @@ -150,12 +176,11 @@ public void testRunWithDummyEnvironmentVariableFails() { pipeline.run(); } - /** - * TestMatcher is a matcher designed for testing matcher serialization/deserialization. - */ + /** TestMatcher is a matcher designed for testing matcher serialization/deserialization. */ public static class TestMatcher extends BaseMatcher implements SerializableMatcher { private final UUID uuid = UUID.randomUUID(); + @Override public boolean matches(Object o) { return true; @@ -180,4 +205,124 @@ public int hashCode() { return uuid.hashCode(); } } + + private static class DummyRunner extends PipelineRunner { + + @SuppressWarnings("unused") // used by reflection + public static DummyRunner fromOptions(final PipelineOptions opts) { + return new DummyRunner(); + } + + @Override + public PipelineResult run(final Pipeline pipeline) { + return new PipelineResult() { + + @Override + public State getState() { + return null; + } + + @Override + public State cancel() throws IOException { + return null; + } + + @Override + public State waitUntilFinish(final Duration duration) { + return null; + } + + @Override + public State waitUntilFinish() { + return null; + } + + @Override + public AggregatorValues getAggregatorValues(final Aggregator aggregator) + throws AggregatorRetrievalException { + return null; + } + + @Override + public MetricResults metrics() { + return null; + } + }; + } + } + + private static PipelineOptions pipelineOptions() { + final PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); + pipelineOptions.setRunner(DummyRunner.class); + return pipelineOptions; + } + + private PCollection pCollection() { + return addTransform(pipeline.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))); + } + + private PCollection addTransform(final PCollection pCollection) { + return pCollection.apply( + MapElements.via( + new SimpleFunction() { + + @Override + public String apply(final String input) { + return DUMMY; + } + })); + } + + @Test + public void testPipelineRunMissing() throws Throwable { + exception.expect(TestPipeline.PipelineRunMissingException.class); + PAssert.that(pCollection()).containsInAnyOrder(DUMMY); + // missing pipeline#run + } + + @Test + public void testPipelineHasAbandonedPAssertNode() throws Throwable { + exception.expect(TestPipeline.AbandonedNodeException.class); + exception.expectMessage("PAssert"); + + final PCollection pCollection = pCollection(); + PAssert.that(pCollection).containsInAnyOrder(DUMMY); + pipeline.run().waitUntilFinish(); + + // dangling PAssert + PAssert.that(pCollection).containsInAnyOrder(DUMMY); + } + + @Test + public void testPipelineHasAbandonedPTransformNode() throws Throwable { + exception.expect(TestPipeline.AbandonedNodeException.class); + exception.expectMessage("PTransform"); + + final PCollection pCollection = pCollection(); + PAssert.that(pCollection).containsInAnyOrder(DUMMY); + pipeline.run().waitUntilFinish(); + + // dangling PTransform + addTransform(pCollection); + } + + @Test + public void testNormalFlowWithPAssert() throws Throwable { + PAssert.that(pCollection()).containsInAnyOrder(DUMMY); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testAutoAddMissingRunFlow() throws Throwable { + PAssert.that(pCollection()).containsInAnyOrder(DUMMY); + // missing pipeline#run, but have it auto-added. + pipeline.enableAutoRunIfMissing(true); + } + + @Test + public void testDisableStrictPAssertFlow() throws Throwable { + pCollection(); + // dangling PTransform, but ignore it + pipeline.enableAbandonedNodeEnforcement(false); + } } From 0d54a4c6708516170cb045b5e0d1d33443973c6e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Sat, 17 Dec 2016 13:57:07 -0800 Subject: [PATCH 229/279] Empty TestPipeline need not be run --- .../apache/beam/sdk/testing/TestPipeline.java | 20 ++++++++++++++++++- .../beam/sdk/testing/TestPipelineTest.java | 3 +++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 49ac3afd0f8a..b707a81deec5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -169,7 +169,12 @@ private void verifyPipelineExecution() { throw new AbandonedNodeException("The pipeline contains abandoned PTransform(s)."); } } else if (runVisitedNodes == null && !enableAutoRunIfMissing) { - throw new PipelineRunMissingException("The pipeline has not been run."); + IsEmptyVisitor isEmptyVisitor = new IsEmptyVisitor(); + pipeline.traverseTopologically(isEmptyVisitor); + + if (!isEmptyVisitor.isEmpty()) { + throw new PipelineRunMissingException("The pipeline has not been run."); + } } } @@ -389,4 +394,17 @@ private static Optional findCallersStackTrace() { } return firstInstanceAfterTestPipeline; } + + private static class IsEmptyVisitor extends PipelineVisitor.Defaults { + private boolean empty = true; + + public boolean isEmpty() { + return empty; + } + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + empty = false; + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java index d1797e76005c..f4845669a636 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java @@ -73,6 +73,9 @@ public class TestPipelineTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); @Rule public transient RuleChain ruleOrder = RuleChain.outerRule(exception).around(pipeline); + @Test + public void testNoTestPipelineUsed() { } + @Test public void testCreationUsingDefaults() { assertNotNull(TestPipeline.create()); From fbfea5953000bd77b07b6a1f1c7a192e24b88021 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 18 Dec 2016 16:02:41 +0100 Subject: [PATCH 230/279] Fix grammar error (repeated for) --- .../org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index d72cea40a416..b6b764e0e423 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -306,7 +306,7 @@ private static DoFnSignature parseSignature(Class> fnClass) String id = onTimerMethod.getAnnotation(DoFn.OnTimer.class).value(); errors.checkArgument( fnContext.getTimerDeclarations().containsKey(id), - "Callback %s is for for undeclared timer %s", + "Callback %s is for undeclared timer %s", onTimerMethod, id); From caf1c720f66de4d502f79b6c11c64b49c53329b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Sun, 11 Dec 2016 07:43:41 +0100 Subject: [PATCH 231/279] [BEAM-716] Use AutoValue in JmsIO --- sdks/java/io/jms/pom.xml | 7 + .../org/apache/beam/sdk/io/jms/JmsIO.java | 321 ++++++++++++------ 2 files changed, 228 insertions(+), 100 deletions(-) diff --git a/sdks/java/io/jms/pom.xml b/sdks/java/io/jms/pom.xml index bca0152f73b4..b88254ed8c2c 100644 --- a/sdks/java/io/jms/pom.xml +++ b/sdks/java/io/jms/pom.xml @@ -81,6 +81,13 @@ jsr305 + + + com.google.auto.value + auto-value + provided + + org.apache.activemq diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 24fa67dbdda3..76dee67bd32b 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; @@ -101,37 +102,148 @@ public class JmsIO { private static final Logger LOG = LoggerFactory.getLogger(JmsIO.class); public static Read read() { - return new Read(null, null, null, Long.MAX_VALUE, null); + return new AutoValue_JmsIO_Read.Builder().setMaxNumRecords(Long.MAX_VALUE).build(); } public static Write write() { - return new Write(null, null, null); + return new AutoValue_JmsIO_Write.Builder().build(); } /** * A {@link PTransform} to read from a JMS destination. See {@link JmsIO} for more * information on usage and configuration. */ - public static class Read extends PTransform> { + @AutoValue + public abstract static class Read extends PTransform> { + /** + * NB: According to http://docs.oracle.com/javaee/1.4/api/javax/jms/ConnectionFactory.html + * "It is expected that JMS providers will provide the tools an administrator needs to create + * and configure administered objects in a JNDI namespace. JMS provider implementations of + * administered objects should be both javax.jndi.Referenceable and java.io.Serializable so + * that they can be stored in all JNDI naming contexts. In addition, it is recommended that + * these implementations follow the JavaBeansTM design patterns." + * + *

      So, a {@link ConnectionFactory} implementation is serializable. + */ + @Nullable abstract ConnectionFactory getConnectionFactory(); + @Nullable abstract String getQueue(); + @Nullable abstract String getTopic(); + abstract long getMaxNumRecords(); + @Nullable abstract Duration getMaxReadTime(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setConnectionFactory(ConnectionFactory connectionFactory); + abstract Builder setQueue(String queue); + abstract Builder setTopic(String topic); + abstract Builder setMaxNumRecords(long maxNumRecords); + abstract Builder setMaxReadTime(Duration maxReadTime); + abstract Read build(); + } + + /** + *

      Specify the JMS connection factory to connect to the JMS broker. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    pipeline.apply(JmsIO.read().withConnectionFactory(myConnectionFactory)
      +     *   }
      +     * 
      + * + * @param connectionFactory The JMS {@link ConnectionFactory}. + * @return The corresponding {@link JmsIO.Read}. + */ public Read withConnectionFactory(ConnectionFactory connectionFactory) { - return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime); + return builder().setConnectionFactory(connectionFactory).build(); } + /** + *

      Specify the JMS queue destination name where to read messages from. The + * {@link JmsIO.Read} acts as a consumer on the queue. + * + *

      This method is exclusive with {@link JmsIO.Read#withTopic(String)}. The user has to + * specify a destination: queue or topic. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    pipeline.apply(JmsIO.read().withQueue("my-queue")
      +     *   }
      +     * 
      + * + * @param queue The JMS queue name where to read messages from. + * @return The corresponding {@link JmsIO.Read}. + */ public Read withQueue(String queue) { - return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime); + return builder().setQueue(queue).build(); } + /** + *

      Specify the JMS topic destination name where to receive messages from. The + * {@link JmsIO.Read} acts as a subscriber on the topic. + * + *

      This method is exclusive with {@link JmsIO.Read#withQueue(String)}. The user has to + * specify a destination: queue or topic. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    pipeline.apply(JmsIO.read().withTopic("my-topic")
      +     *   }
      +     * 
      + * + * @param topic The JMS topic name. + * @return The corresponding {@link JmsIO.Read}. + */ public Read withTopic(String topic) { - return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime); + return builder().setTopic(topic).build(); } + /** + *

      Define the max number of records that the source will read. Using a max number of records + * different from {@code Long.MAX_VALUE} means the source will be {@code Bounded}, and will + * stop once the max number of records read is reached. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    pipeline.apply(JmsIO.read().withNumRecords(1000)
      +     *   }
      +     * 
      + * + * @param maxNumRecords The max number of records to read from the JMS destination. + * @return The corresponding {@link JmsIO.Read}. + */ public Read withMaxNumRecords(long maxNumRecords) { - return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime); + return builder().setMaxNumRecords(maxNumRecords).build(); } + /** + *

      Define the max read time that the source will read. Using a non null max read time + * duration means the source will be {@code Bounded}, and will stop once the max read time is + * reached. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    pipeline.apply(JmsIO.read().withMaxReadTime(Duration.minutes(10))
      +     *   }
      +     * 
      + * + * @param maxReadTime The max read time duration. + * @return The corresponding {@link JmsIO.Read}. + */ public Read withMaxReadTime(Duration maxReadTime) { - return new Read(connectionFactory, queue, topic, maxNumRecords, maxReadTime); + return builder().setMaxReadTime(maxReadTime).build(); } @Override @@ -141,10 +253,10 @@ public PCollection expand(PBegin input) { PTransform> transform = unbounded; - if (maxNumRecords != Long.MAX_VALUE) { - transform = unbounded.withMaxNumRecords(maxNumRecords); - } else if (maxReadTime != null) { - transform = unbounded.withMaxReadTime(maxReadTime); + if (getMaxNumRecords() != Long.MAX_VALUE) { + transform = unbounded.withMaxNumRecords(getMaxNumRecords()); + } else if (getMaxReadTime() != null) { + transform = unbounded.withMaxReadTime(getMaxReadTime()); } return input.getPipeline().apply(transform); @@ -152,54 +264,21 @@ public PCollection expand(PBegin input) { @Override public void validate(PBegin input) { - checkNotNull(connectionFactory, "ConnectionFactory not specified"); - checkArgument((queue != null || topic != null), "Either queue or topic not specified"); + checkNotNull(getConnectionFactory(), "ConnectionFactory not specified"); + checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic not " + + "specified"); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - builder.addIfNotNull(DisplayData.item("queue", queue)); - builder.addIfNotNull(DisplayData.item("topic", topic)); + builder.addIfNotNull(DisplayData.item("queue", getQueue())); + builder.addIfNotNull(DisplayData.item("topic", getTopic())); } /////////////////////////////////////////////////////////////////////////////////////// - /** - * NB: According to http://docs.oracle.com/javaee/1.4/api/javax/jms/ConnectionFactory.html - * "It is expected that JMS providers will provide the tools an administrator needs to create - * and configure administered objects in a JNDI namespace. JMS provider implementations of - * administered objects should be both javax.jndi.Referenceable and java.io.Serializable so - * that they can be stored in all JNDI naming contexts. In addition, it is recommended that - * these implementations follow the JavaBeansTM design patterns." - * - *

      So, a {@link ConnectionFactory} implementation is serializable. - */ - protected ConnectionFactory connectionFactory; - @Nullable - protected String queue; - @Nullable - protected String topic; - protected long maxNumRecords; - protected Duration maxReadTime; - - private Read( - ConnectionFactory connectionFactory, - String queue, - String topic, - long maxNumRecords, - Duration maxReadTime) { - super("JmsIO.Read"); - - this.connectionFactory = connectionFactory; - this.queue = queue; - this.topic = topic; - this.maxNumRecords = maxNumRecords; - this.maxReadTime = maxReadTime; - } - /** * Creates an {@link UnboundedSource UnboundedSource<JmsRecord, ?>} with the configuration * in {@link Read}. Primary use case is unit tests, should not be used in an @@ -207,10 +286,7 @@ private Read( */ @VisibleForTesting UnboundedSource createSource() { - return new UnboundedJmsSource( - connectionFactory, - queue, - topic); + return new UnboundedJmsSource(this); } } @@ -219,17 +295,10 @@ private JmsIO() {} private static class UnboundedJmsSource extends UnboundedSource { - private final ConnectionFactory connectionFactory; - private final String queue; - private final String topic; + private final Read spec; - public UnboundedJmsSource( - ConnectionFactory connectionFactory, - String queue, - String topic) { - this.connectionFactory = connectionFactory; - this.queue = queue; - this.topic = topic; + public UnboundedJmsSource(Read spec) { + this.spec = spec; } @Override @@ -237,7 +306,7 @@ public List generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { List sources = new ArrayList<>(); for (int i = 0; i < desiredNumSplits; i++) { - sources.add(new UnboundedJmsSource(connectionFactory, queue, topic)); + sources.add(new UnboundedJmsSource(spec)); } return sources; } @@ -250,8 +319,7 @@ public UnboundedJmsReader createReader(PipelineOptions options, @Override public void validate() { - checkNotNull(connectionFactory, "ConnectionFactory is not defined"); - checkArgument((queue != null || topic != null), "Either queue or topic is not defined"); + spec.validate(null); } @Override @@ -291,15 +359,17 @@ public UnboundedJmsReader( @Override public boolean start() throws IOException { - ConnectionFactory connectionFactory = source.connectionFactory; + ConnectionFactory connectionFactory = source.spec.getConnectionFactory(); try { this.connection = connectionFactory.createConnection(); this.connection.start(); this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE); - if (source.topic != null) { - this.consumer = this.session.createConsumer(this.session.createTopic(source.topic)); + if (source.spec.getTopic() != null) { + this.consumer = + this.session.createConsumer(this.session.createTopic(source.spec.getTopic())); } else { - this.consumer = this.session.createConsumer(this.session.createQueue(source.queue)); + this.consumer = + this.session.createConsumer(this.session.createQueue(source.spec.getQueue())); } return advance(); @@ -409,70 +479,122 @@ public void close() throws IOException { * A {@link PTransform} to write to a JMS queue. See {@link JmsIO} for * more information on usage and configuration. */ - public static class Write extends PTransform, PDone> { + @AutoValue + public abstract static class Write extends PTransform, PDone> { - protected ConnectionFactory connectionFactory; - protected String queue; - protected String topic; + @Nullable abstract ConnectionFactory getConnectionFactory(); + @Nullable abstract String getQueue(); + @Nullable abstract String getTopic(); + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setConnectionFactory(ConnectionFactory connectionFactory); + abstract Builder setQueue(String queue); + abstract Builder setTopic(String topic); + abstract Write build(); + } + + /** + *

      Specify the JMS connection factory to connect to the JMS broker. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    .apply(JmsIO.write().withConnectionFactory(myConnectionFactory)
      +     *   }
      +     * 
      + * + * @param connectionFactory The JMS {@link ConnectionFactory}. + * @return The corresponding {@link JmsIO.Read}. + */ public Write withConnectionFactory(ConnectionFactory connectionFactory) { - return new Write(connectionFactory, queue, topic); + return builder().setConnectionFactory(connectionFactory).build(); } + /** + *

      Specify the JMS queue destination name where to send messages to. The + * {@link JmsIO.Write} acts as a producer on the queue. + * + *

      This method is exclusive with {@link JmsIO.Write#withTopic(String)}. The user has to + * specify a destination: queue or topic. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    .apply(JmsIO.write().withQueue("my-queue")
      +     *   }
      +     * 
      + * + * @param queue The JMS queue name where to send messages to. + * @return The corresponding {@link JmsIO.Read}. + */ public Write withQueue(String queue) { - return new Write(connectionFactory, queue, topic); + return builder().setQueue(queue).build(); } + /** + *

      Specify the JMS topic destination name where to send messages to. The + * {@link JmsIO.Read} acts as a publisher on the topic. + * + *

      This method is exclusive with {@link JmsIO.Write#withQueue(String)}. The user has to + * specify a destination: queue or topic. + * + *

      For instance: + * + *

      +     *   {@code
      +     *    .apply(JmsIO.write().withTopic("my-topic")
      +     *   }
      +     * 
      + * + * @param topic The JMS topic name. + * @return The corresponding {@link JmsIO.Read}. + */ public Write withTopic(String topic) { - return new Write(connectionFactory, queue, topic); - } - - private Write(ConnectionFactory connectionFactory, String queue, String topic) { - this.connectionFactory = connectionFactory; - this.queue = queue; - this.topic = topic; + return builder().setTopic(topic).build(); } @Override public PDone expand(PCollection input) { - input.apply(ParDo.of(new JmsWriter(connectionFactory, queue, topic))); + input.apply(ParDo.of(new WriterFn(this))); return PDone.in(input.getPipeline()); } @Override public void validate(PCollection input) { - checkNotNull(connectionFactory, "ConnectionFactory is not defined"); - checkArgument((queue != null || topic != null), "Either queue or topic is required"); + checkNotNull(getConnectionFactory(), "ConnectionFactory is not defined"); + checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic is " + + "required"); } - private static class JmsWriter extends DoFn { + private static class WriterFn extends DoFn { - private ConnectionFactory connectionFactory; - private String queue; - private String topic; + private Write spec; private Connection connection; private Session session; private MessageProducer producer; - public JmsWriter(ConnectionFactory connectionFactory, String queue, String topic) { - this.connectionFactory = connectionFactory; - this.queue = queue; - this.topic = topic; + public WriterFn(Write spec) { + this.spec = spec; } @StartBundle public void startBundle(Context c) throws Exception { if (producer == null) { - this.connection = connectionFactory.createConnection(); + this.connection = spec.getConnectionFactory().createConnection(); this.connection.start(); // false means we don't use JMS transaction. this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE); Destination destination; - if (queue != null) { - destination = session.createQueue(queue); + if (spec.getQueue() != null) { + destination = session.createQueue(spec.getQueue()); } else { - destination = session.createTopic(topic); + destination = session.createTopic(spec.getTopic()); } this.producer = this.session.createProducer(destination); } @@ -481,7 +603,6 @@ public void startBundle(Context c) throws Exception { @ProcessElement public void processElement(ProcessContext ctx) throws Exception { String value = ctx.element(); - try { TextMessage message = session.createTextMessage(value); producer.send(message); From 30e14cfa63db50d567185599ea049c96229b48e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 13 Dec 2016 21:55:46 +0100 Subject: [PATCH 232/279] [BEAM-716] Fix javadoc on with* methods [BEAM-959] Improve check preconditions in JmsIO --- .../org/apache/beam/sdk/io/jms/JmsIO.java | 45 ++++++++++++------- 1 file changed, 30 insertions(+), 15 deletions(-) diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 76dee67bd32b..b6de26a0c2ae 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.io.jms; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; @@ -145,7 +145,7 @@ abstract static class Builder { } /** - *

      Specify the JMS connection factory to connect to the JMS broker. + * Specify the JMS connection factory to connect to the JMS broker. * *

      For instance: * @@ -159,11 +159,13 @@ abstract static class Builder { * @return The corresponding {@link JmsIO.Read}. */ public Read withConnectionFactory(ConnectionFactory connectionFactory) { + checkArgument(connectionFactory != null, "withConnectionFactory(connectionFactory) called" + + " with null connectionFactory"); return builder().setConnectionFactory(connectionFactory).build(); } /** - *

      Specify the JMS queue destination name where to read messages from. The + * Specify the JMS queue destination name where to read messages from. The * {@link JmsIO.Read} acts as a consumer on the queue. * *

      This method is exclusive with {@link JmsIO.Read#withTopic(String)}. The user has to @@ -181,11 +183,12 @@ public Read withConnectionFactory(ConnectionFactory connectionFactory) { * @return The corresponding {@link JmsIO.Read}. */ public Read withQueue(String queue) { + checkArgument(queue != null, "withQueue(queue) called with null queue"); return builder().setQueue(queue).build(); } /** - *

      Specify the JMS topic destination name where to receive messages from. The + * Specify the JMS topic destination name where to receive messages from. The * {@link JmsIO.Read} acts as a subscriber on the topic. * *

      This method is exclusive with {@link JmsIO.Read#withQueue(String)}. The user has to @@ -203,11 +206,12 @@ public Read withQueue(String queue) { * @return The corresponding {@link JmsIO.Read}. */ public Read withTopic(String topic) { + checkArgument(topic != null, "withTopic(topic) called with null topic"); return builder().setTopic(topic).build(); } /** - *

      Define the max number of records that the source will read. Using a max number of records + * Define the max number of records that the source will read. Using a max number of records * different from {@code Long.MAX_VALUE} means the source will be {@code Bounded}, and will * stop once the max number of records read is reached. * @@ -223,11 +227,13 @@ public Read withTopic(String topic) { * @return The corresponding {@link JmsIO.Read}. */ public Read withMaxNumRecords(long maxNumRecords) { + checkArgument(maxNumRecords >= 0, "withMaxNumRecords(maxNumRecords) called with invalid " + + "maxNumRecords"); return builder().setMaxNumRecords(maxNumRecords).build(); } /** - *

      Define the max read time that the source will read. Using a non null max read time + * Define the max read time that the source will read. Using a non null max read time * duration means the source will be {@code Bounded}, and will stop once the max read time is * reached. * @@ -243,6 +249,8 @@ public Read withMaxNumRecords(long maxNumRecords) { * @return The corresponding {@link JmsIO.Read}. */ public Read withMaxReadTime(Duration maxReadTime) { + checkArgument(maxReadTime != null, "withMaxReadTime(maxReadTime) called with null " + + "maxReadTime"); return builder().setMaxReadTime(maxReadTime).build(); } @@ -264,9 +272,11 @@ public PCollection expand(PBegin input) { @Override public void validate(PBegin input) { - checkNotNull(getConnectionFactory(), "ConnectionFactory not specified"); - checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic not " - + "specified"); + checkState(getConnectionFactory() != null, "JmsIO.read() requires a JMS connection " + + "factory to be set via withConnectionFactory(connectionFactory)"); + checkState((getQueue() != null || getTopic() != null), "JmsIO.read() requires a JMS " + + "destination (queue or topic) to be set via withQueue(queueName) or withTopic" + + "(topicName)"); } @Override @@ -497,7 +507,7 @@ abstract static class Builder { } /** - *

      Specify the JMS connection factory to connect to the JMS broker. + * Specify the JMS connection factory to connect to the JMS broker. * *

      For instance: * @@ -511,11 +521,13 @@ abstract static class Builder { * @return The corresponding {@link JmsIO.Read}. */ public Write withConnectionFactory(ConnectionFactory connectionFactory) { + checkArgument(connectionFactory != null, "withConnectionFactory(connectionFactory) called" + + " with null connectionFactory"); return builder().setConnectionFactory(connectionFactory).build(); } /** - *

      Specify the JMS queue destination name where to send messages to. The + * Specify the JMS queue destination name where to send messages to. The * {@link JmsIO.Write} acts as a producer on the queue. * *

      This method is exclusive with {@link JmsIO.Write#withTopic(String)}. The user has to @@ -533,11 +545,12 @@ public Write withConnectionFactory(ConnectionFactory connectionFactory) { * @return The corresponding {@link JmsIO.Read}. */ public Write withQueue(String queue) { + checkArgument(queue != null, "withQueue(queue) called with null queue"); return builder().setQueue(queue).build(); } /** - *

      Specify the JMS topic destination name where to send messages to. The + * Specify the JMS topic destination name where to send messages to. The * {@link JmsIO.Read} acts as a publisher on the topic. * *

      This method is exclusive with {@link JmsIO.Write#withQueue(String)}. The user has to @@ -555,6 +568,7 @@ public Write withQueue(String queue) { * @return The corresponding {@link JmsIO.Read}. */ public Write withTopic(String topic) { + checkArgument(topic != null, "withTopic(topic) called with null topic"); return builder().setTopic(topic).build(); } @@ -566,9 +580,10 @@ public PDone expand(PCollection input) { @Override public void validate(PCollection input) { - checkNotNull(getConnectionFactory(), "ConnectionFactory is not defined"); - checkArgument((getQueue() != null || getTopic() != null), "Either queue or topic is " - + "required"); + checkState(getConnectionFactory() != null, "JmsIO.write() requires a JMS connection " + + "factory to be set via withConnectionFactory(connectionFactory)"); + checkState((getQueue() != null || getTopic() != null), "JmsIO.write() requires a JMS " + + "destination (queue or topic) to be set via withQueue(queue) or withTopic(topic)"); } private static class WriterFn extends DoFn { From 4cf59170546da3689fca4352ccde259cee408331 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 18 Dec 2016 21:01:13 +0100 Subject: [PATCH 233/279] [BEAM-1178] Make naming of logger objects consistent --- .../beam/sdk/io/range/ByteKeyRange.java | 4 +- .../sdk/io/range/ByteKeyRangeTracker.java | 8 +-- .../beam/sdk/metrics/MetricsEnvironment.java | 6 +-- .../org/apache/beam/sdk/util/ApiSurface.java | 52 +++++++++---------- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 24 ++++----- .../io/gcp/bigtable/BigtableServiceImpl.java | 6 +-- .../apache/beam/sdk/io/jdbc/JdbcIOTest.java | 4 +- .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 12 ++--- .../sdk/io/mongodb/MongoDBGridFSIOTest.java | 8 +-- .../beam/sdk/io/mongodb/MongoDbIOTest.java | 8 +-- 10 files changed, 66 insertions(+), 66 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java index e30f8af911b0..0212e8a3a20f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRange.java @@ -69,7 +69,7 @@ * @see ByteKey */ public final class ByteKeyRange implements Serializable { - private static final Logger logger = LoggerFactory.getLogger(ByteKeyRange.class); + private static final Logger LOG = LoggerFactory.getLogger(ByteKeyRange.class); /** The range of all keys, with empty start and end keys. */ public static final ByteKeyRange ALL_KEYS = ByteKeyRange.of(ByteKey.EMPTY, ByteKey.EMPTY); @@ -191,7 +191,7 @@ public double estimateFractionForKey(ByteKey key) { // Keys are equal subject to padding by 0. BigInteger range = rangeEndInt.subtract(rangeStartInt); if (range.equals(BigInteger.ZERO)) { - logger.warn( + LOG.warn( "Using 0.0 as the default fraction for this near-empty range {} where start and end keys" + " differ only by trailing zeros.", this); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java index 7c0f1c0a6d38..99717a4bffa9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java @@ -32,7 +32,7 @@ * @see ByteKeyRange */ public final class ByteKeyRangeTracker implements RangeTracker { - private static final Logger logger = LoggerFactory.getLogger(ByteKeyRangeTracker.class); + private static final Logger LOG = LoggerFactory.getLogger(ByteKeyRangeTracker.class); /** Instantiates a new {@link ByteKeyRangeTracker} with the specified range. */ public static ByteKeyRangeTracker of(ByteKeyRange range) { @@ -89,7 +89,7 @@ public synchronized boolean tryReturnRecordAt(boolean isAtSplitPoint, ByteKey re public synchronized boolean trySplitAtPosition(ByteKey splitPosition) { // Unstarted. if (position == null) { - logger.warn( + LOG.warn( "{}: Rejecting split request at {} because no records have been returned.", this, splitPosition); @@ -98,7 +98,7 @@ public synchronized boolean trySplitAtPosition(ByteKey splitPosition) { // Started, but not after current position. if (splitPosition.compareTo(position) <= 0) { - logger.warn( + LOG.warn( "{}: Rejecting split request at {} because it is not after current position {}.", this, splitPosition, @@ -108,7 +108,7 @@ public synchronized boolean trySplitAtPosition(ByteKey splitPosition) { // Sanity check. if (!range.containsKey(splitPosition)) { - logger.warn( + LOG.warn( "{}: Rejecting split request at {} because it is not within the range.", this, splitPosition); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java index 5d7cb0b1c093..2942578b1e95 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java @@ -41,7 +41,7 @@ */ public class MetricsEnvironment { - private static final Logger LOGGER = LoggerFactory.getLogger(MetricsContainer.class); + private static final Logger LOG = LoggerFactory.getLogger(MetricsContainer.class); private static final AtomicBoolean METRICS_SUPPORTED = new AtomicBoolean(false); private static final AtomicBoolean REPORTED_MISSING_CONTAINER = new AtomicBoolean(false); @@ -107,11 +107,11 @@ public static MetricsContainer getCurrentContainer() { MetricsContainer container = CONTAINER_FOR_THREAD.get(); if (container == null && REPORTED_MISSING_CONTAINER.compareAndSet(false, true)) { if (METRICS_SUPPORTED.get()) { - LOGGER.error( + LOG.error( "Unable to update metrics on the current thread. " + "Most likely caused by using metrics outside the managed work-execution thread."); } else { - LOGGER.warn("Reporting metrics are not supported in the current execution environment."); + LOG.warn("Reporting metrics are not supported in the current execution environment."); } } return container; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java index 3914bb0e10a1..2040161a708c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java @@ -75,13 +75,13 @@ */ @SuppressWarnings("rawtypes") public class ApiSurface { - private static Logger logger = LoggerFactory.getLogger(ApiSurface.class); + private static final Logger LOG = LoggerFactory.getLogger(ApiSurface.class); /** * Returns an empty {@link ApiSurface}. */ public static ApiSurface empty() { - logger.debug("Returning an empty ApiSurface"); + LOG.debug("Returning an empty ApiSurface"); return new ApiSurface(Collections.>emptySet(), Collections.emptySet()); } @@ -113,7 +113,7 @@ public ApiSurface includingPackage(String packageName) throws IOException { newRootClasses.add(clazz); } } - logger.debug("Including package {} and subpackages: {}", packageName, newRootClasses); + LOG.debug("Including package {} and subpackages: {}", packageName, newRootClasses); newRootClasses.addAll(rootClasses); return new ApiSurface(newRootClasses, patternsToPrune); @@ -124,7 +124,7 @@ public ApiSurface includingPackage(String packageName) throws IOException { */ public ApiSurface includingClass(Class clazz) { Set> newRootClasses = Sets.newHashSet(); - logger.debug("Including class {}", clazz); + LOG.debug("Including class {}", clazz); newRootClasses.add(clazz); newRootClasses.addAll(rootClasses); return new ApiSurface(newRootClasses, patternsToPrune); @@ -360,7 +360,7 @@ private void visit(Type type) { * See {@link #addExposedTypes(Type, Class)}. */ private void addExposedTypes(TypeToken type, Class cause) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is the type in type token {}", type.getType(), type); addExposedTypes(type.getType(), cause); } @@ -372,19 +372,19 @@ private void addExposedTypes(TypeToken type, Class cause) { */ private void addExposedTypes(Type type, Class cause) { if (type instanceof TypeVariable) { - logger.debug("Adding exposed types from {}, which is a type variable", type); + LOG.debug("Adding exposed types from {}, which is a type variable", type); addExposedTypes((TypeVariable) type, cause); } else if (type instanceof WildcardType) { - logger.debug("Adding exposed types from {}, which is a wildcard type", type); + LOG.debug("Adding exposed types from {}, which is a wildcard type", type); addExposedTypes((WildcardType) type, cause); } else if (type instanceof GenericArrayType) { - logger.debug("Adding exposed types from {}, which is a generic array type", type); + LOG.debug("Adding exposed types from {}, which is a generic array type", type); addExposedTypes((GenericArrayType) type, cause); } else if (type instanceof ParameterizedType) { - logger.debug("Adding exposed types from {}, which is a parameterized type", type); + LOG.debug("Adding exposed types from {}, which is a parameterized type", type); addExposedTypes((ParameterizedType) type, cause); } else if (type instanceof Class) { - logger.debug("Adding exposed types from {}, which is a class", type); + LOG.debug("Adding exposed types from {}, which is a class", type); addExposedTypes((Class) type, cause); } else { throw new IllegalArgumentException("Unknown implementation of Type"); @@ -402,7 +402,7 @@ private void addExposedTypes(TypeVariable type, Class cause) { } visit(type); for (Type bound : type.getBounds()) { - logger.debug("Adding exposed types from {}, which is a type bound on {}", bound, type); + LOG.debug("Adding exposed types from {}, which is a type bound on {}", bound, type); addExposedTypes(bound, cause); } } @@ -414,14 +414,14 @@ private void addExposedTypes(TypeVariable type, Class cause) { private void addExposedTypes(WildcardType type, Class cause) { visit(type); for (Type lowerBound : type.getLowerBounds()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is a type lower bound on wildcard type {}", lowerBound, type); addExposedTypes(lowerBound, cause); } for (Type upperBound : type.getUpperBounds()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is a type upper bound on wildcard type {}", upperBound, type); @@ -439,7 +439,7 @@ private void addExposedTypes(GenericArrayType type, Class cause) { return; } visit(type); - logger.debug( + LOG.debug( "Adding exposed types from {}, which is the component type on generic array type {}", type.getGenericComponentType(), type); @@ -467,13 +467,13 @@ private void addExposedTypes(ParameterizedType type, Class cause) { // The type parameters themselves may not be pruned, // for example with List probably the // standard List is pruned, but MyApiType is not. - logger.debug( + LOG.debug( "Adding exposed types from {}, which is the raw type on parameterized type {}", type.getRawType(), type); addExposedTypes(type.getRawType(), cause); for (Type typeArg : type.getActualTypeArguments()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is a type argument on parameterized type {}", typeArg, type); @@ -501,14 +501,14 @@ private void addExposedTypes(Class clazz, Class cause) { TypeToken token = TypeToken.of(clazz); for (TypeToken superType : token.getTypes()) { if (!superType.equals(token)) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is a super type token on {}", superType, clazz); addExposedTypes(superType, clazz); } } for (Class innerClass : clazz.getDeclaredClasses()) { if (exposed(innerClass.getModifiers())) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is an exposed inner class of {}", innerClass, clazz); @@ -517,12 +517,12 @@ private void addExposedTypes(Class clazz, Class cause) { } for (Field field : clazz.getDeclaredFields()) { if (exposed(field.getModifiers())) { - logger.debug("Adding exposed types from {}, which is an exposed field on {}", field, clazz); + LOG.debug("Adding exposed types from {}, which is an exposed field on {}", field, clazz); addExposedTypes(field, clazz); } } for (Invokable invokable : getExposedInvokables(token)) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is an exposed invokable on {}", invokable, clazz); addExposedTypes(invokable, clazz); } @@ -531,21 +531,21 @@ private void addExposedTypes(Class clazz, Class cause) { private void addExposedTypes(Invokable invokable, Class cause) { addExposedTypes(invokable.getReturnType(), cause); for (Annotation annotation : invokable.getAnnotations()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is an annotation on invokable {}", annotation, invokable); addExposedTypes(annotation.annotationType(), cause); } for (Parameter parameter : invokable.getParameters()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is a parameter on invokable {}", parameter, invokable); addExposedTypes(parameter, cause); } for (TypeToken exceptionType : invokable.getExceptionTypes()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is an exception type on invokable {}", exceptionType, invokable); @@ -554,13 +554,13 @@ private void addExposedTypes(Invokable invokable, Class cause) { } private void addExposedTypes(Parameter parameter, Class cause) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is the type of parameter {}", parameter.getType(), parameter); addExposedTypes(parameter.getType(), cause); for (Annotation annotation : parameter.getAnnotations()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is an annotation on parameter {}", annotation, parameter); @@ -571,7 +571,7 @@ private void addExposedTypes(Parameter parameter, Class cause) { private void addExposedTypes(Field field, Class cause) { addExposedTypes(field.getGenericType(), cause); for (Annotation annotation : field.getDeclaredAnnotations()) { - logger.debug( + LOG.debug( "Adding exposed types from {}, which is an annotation on field {}", annotation, field); addExposedTypes(annotation.annotationType(), cause); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index a83784babbe0..c27389da36e9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -154,7 +154,7 @@ */ @Experimental public class BigtableIO { - private static final Logger logger = LoggerFactory.getLogger(BigtableIO.class); + private static final Logger LOG = LoggerFactory.getLogger(BigtableIO.class); /** * Creates an uninitialized {@link BigtableIO.Read}. Before use, the {@code Read} must be @@ -307,7 +307,7 @@ public void validate(PBegin input) { "Table %s does not exist", tableId); } catch (IOException e) { - logger.warn("Error checking whether table {} exists; proceeding.", tableId, e); + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); } } @@ -521,7 +521,7 @@ public void validate(PCollection>> input) { "Table %s does not exist", tableId); } catch (IOException e) { - logger.warn("Error checking whether table {} exists; proceeding.", tableId, e); + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); } } @@ -612,7 +612,7 @@ public void processElement(ProcessContext c) throws Exception { public void finishBundle(Context c) throws Exception { bigtableWriter.flush(); checkForFailures(); - logger.info("Wrote {} records", recordsWritten); + LOG.info("Wrote {} records", recordsWritten); } @Teardown @@ -658,7 +658,7 @@ private void checkForFailures() throws IOException { i + failures.size(), i, logEntry.toString()); - logger.error(message); + LOG.error(message); throw new IOException(message); } @@ -762,11 +762,11 @@ private List splitIntoBundlesBasedOnSamples( long desiredBundleSizeBytes, List sampleRowKeys) { // There are no regions, or no samples available. Just scan the entire range. if (sampleRowKeys.isEmpty()) { - logger.info("Not splitting source {} because no sample row keys are available.", this); + LOG.info("Not splitting source {} because no sample row keys are available.", this); return Collections.singletonList(this); } - logger.info( + LOG.info( "About to split into bundles of size {} with sampleRowKeys length {} first element {}", desiredBundleSizeBytes, sampleRowKeys.size(), @@ -832,7 +832,7 @@ private List splitIntoBundlesBasedOnSamples( } List ret = splits.build(); - logger.info("Generated {} splits. First split: {}", ret.size(), ret.get(0)); + LOG.info("Generated {} splits. First split: {}", ret.size(), ret.get(0)); return ret; } @@ -912,7 +912,7 @@ public Coder getDefaultOutputCoder() { private List splitKeyRangeIntoBundleSizedSubranges( long sampleSizeBytes, long desiredBundleSizeBytes, ByteKeyRange range) { // Catch the trivial cases. Split is small enough already, or this is the last region. - logger.debug( + LOG.debug( "Subsplit for sampleSizeBytes {} and desiredBundleSizeBytes {}", sampleSizeBytes, desiredBundleSizeBytes); @@ -1010,7 +1010,7 @@ public Row getCurrent() throws NoSuchElementException { @Override public void close() throws IOException { - logger.info("Closing reader after reading {} records.", recordsReturned); + LOG.info("Closing reader after reading {} records.", recordsReturned); if (reader != null) { reader.close(); reader = null; @@ -1033,11 +1033,11 @@ public final synchronized BigtableSource splitAtFraction(double fraction) { try { splitKey = rangeTracker.getRange().interpolateKey(fraction); } catch (IllegalArgumentException e) { - logger.info( + LOG.info( "%s: Failed to interpolate key for fraction %s.", rangeTracker.getRange(), fraction); return null; } - logger.debug( + LOG.debug( "Proposing to split {} at fraction {} (key {})", rangeTracker, fraction, splitKey); BigtableSource primary = source.withEndKey(splitKey); BigtableSource residual = source.withStartKey(splitKey); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 7ce4b4af9d69..1a4937cbefd3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -52,7 +52,7 @@ * service. */ class BigtableServiceImpl implements BigtableService { - private static final Logger logger = LoggerFactory.getLogger(BigtableService.class); + private static final Logger LOG = LoggerFactory.getLogger(BigtableService.class); public BigtableServiceImpl(BigtableOptions options) { this.options = options; @@ -75,7 +75,7 @@ public BigtableWriterImpl openForWriting(String tableId) throws IOException { @Override public boolean tableExists(String tableId) throws IOException { if (!BigtableSession.isAlpnProviderEnabled()) { - logger.info( + LOG.info( "Skipping existence check for table {} (BigtableOptions {}) because ALPN is not" + " configured.", tableId, @@ -97,7 +97,7 @@ public boolean tableExists(String tableId) throws IOException { String message = String.format( "Error checking whether table %s (BigtableOptions %s) exists", tableId, options); - logger.error(message, e); + LOG.error(message, e); throw new IOException(message, e); } } diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java index d09929ddd17c..aa93a222b6bc 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java @@ -57,7 +57,7 @@ * Test on the JdbcIO. */ public class JdbcIOTest implements Serializable { - private static final Logger LOGGER = LoggerFactory.getLogger(JdbcIOTest.class); + private static final Logger LOG = LoggerFactory.getLogger(JdbcIOTest.class); private static NetworkServerControl derbyServer; private static ClientDataSource dataSource; @@ -70,7 +70,7 @@ public static void startDatabase() throws Exception { port = socket.getLocalPort(); socket.close(); - LOGGER.info("Starting Derby database on {}", port); + LOG.info("Starting Derby database on {}", port); System.setProperty("derby.stream.error.file", "target/derby.log"); diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index f5394319efb7..de2eb161cf10 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -93,7 +93,7 @@ */ public class MongoDbIO { - private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIO.class); + private static final Logger LOG = LoggerFactory.getLogger(MongoDbIO.class); /** Read data from MongoDB. */ public static Read read() { @@ -253,19 +253,19 @@ public List> splitIntoBundles(long desiredBundleSizeByte splitVectorCommand.append("keyPattern", new BasicDBObject().append("_id", 1)); splitVectorCommand.append("force", false); // maxChunkSize is the Mongo partition size in MB - LOGGER.debug("Splitting in chunk of {} MB", desiredBundleSizeBytes / 1024 / 1024); + LOG.debug("Splitting in chunk of {} MB", desiredBundleSizeBytes / 1024 / 1024); splitVectorCommand.append("maxChunkSize", desiredBundleSizeBytes / 1024 / 1024); Document splitVectorCommandResult = mongoDatabase.runCommand(splitVectorCommand); splitKeys = (List) splitVectorCommandResult.get("splitKeys"); List> sources = new ArrayList<>(); if (splitKeys.size() < 1) { - LOGGER.debug("Split keys is low, using an unique source"); + LOG.debug("Split keys is low, using an unique source"); sources.add(this); return sources; } - LOGGER.debug("Number of splits is {}", splitKeys.size()); + LOG.debug("Number of splits is {}", splitKeys.size()); for (String shardFilter : splitKeysToFilters(splitKeys, spec.filter())) { sources.add(new BoundedMongoDbSource(spec.withFilter(shardFilter))); } @@ -392,12 +392,12 @@ public void close() { cursor.close(); } } catch (Exception e) { - LOGGER.warn("Error closing MongoDB cursor", e); + LOG.warn("Error closing MongoDB cursor", e); } try { client.close(); } catch (Exception e) { - LOGGER.warn("Error closing MongoDB client", e); + LOG.warn("Error closing MongoDB client", e); } } diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java index 50614872ee76..df05c934194c 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java @@ -88,7 +88,7 @@ * Test on the MongoDbGridFSIO. */ public class MongoDBGridFSIOTest implements Serializable { - private static final Logger LOGGER = LoggerFactory.getLogger(MongoDBGridFSIOTest.class); + private static final Logger LOG = LoggerFactory.getLogger(MongoDBGridFSIOTest.class); private static final String MONGODB_LOCATION = "target/mongodb"; private static final String DATABASE = "gridfs"; @@ -105,7 +105,7 @@ public static void setup() throws Exception { try (ServerSocket serverSocket = new ServerSocket(0)) { port = serverSocket.getLocalPort(); } - LOGGER.info("Starting MongoDB embedded instance on {}", port); + LOG.info("Starting MongoDB embedded instance on {}", port); try { Files.forceDelete(new File(MONGODB_LOCATION)); } catch (Exception e) { @@ -127,7 +127,7 @@ public static void setup() throws Exception { mongodExecutable = mongodStarter.prepare(mongodConfig); mongodProcess = mongodExecutable.start(); - LOGGER.info("Insert test data"); + LOG.info("Insert test data"); Mongo client = new Mongo("localhost", port); DB database = client.getDB(DATABASE); @@ -174,7 +174,7 @@ public static void setup() throws Exception { @AfterClass public static void stop() throws Exception { - LOGGER.info("Stopping MongoDB instance"); + LOG.info("Stopping MongoDB instance"); mongodProcess.stop(); mongodExecutable.stop(); } diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java index 129e81c02c97..5faa618ba75d 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -67,7 +67,7 @@ */ public class MongoDbIOTest implements Serializable { - private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIOTest.class); + private static final Logger LOG = LoggerFactory.getLogger(MongoDbIOTest.class); private static final String MONGODB_LOCATION = "target/mongodb"; private static final String DATABASE = "beam"; @@ -92,7 +92,7 @@ public static void availablePort() throws Exception { @Before public void setup() throws Exception { - LOGGER.info("Starting MongoDB embedded instance on {}", port); + LOG.info("Starting MongoDB embedded instance on {}", port); try { Files.forceDelete(new File(MONGODB_LOCATION)); } catch (Exception e) { @@ -114,7 +114,7 @@ public void setup() throws Exception { mongodExecutable = mongodStarter.prepare(mongodConfig); mongodProcess = mongodExecutable.start(); - LOGGER.info("Insert test data"); + LOG.info("Insert test data"); MongoClient client = new MongoClient("localhost", port); MongoDatabase database = client.getDatabase(DATABASE); @@ -135,7 +135,7 @@ public void setup() throws Exception { @After public void stop() throws Exception { - LOGGER.info("Stopping MongoDB instance"); + LOG.info("Stopping MongoDB instance"); mongodProcess.stop(); mongodExecutable.stop(); } From 9bd4288f13ac3f279b795e93adcfa3f897ef266c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 15 Dec 2016 22:06:18 +0100 Subject: [PATCH 234/279] [BEAM-1165] Fix unexpected file creation when checking dependencies This error happens because maven-dependency-plugin asumes the property output used by the flink tests as the export file for the depedency:tree command. Ref. https://maven.apache.org/plugins/maven-dependency-plugin/tree-mojo.html#output --- runners/flink/examples/pom.xml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index 314584870a9e..c50a2937c0dd 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -34,9 +34,9 @@ - kinglear.txt - wordcounts.txt - -1 + kinglear.txt + wordcounts.txt + -1 @@ -112,9 +112,9 @@ java --runner=org.apache.beam.runners.flink.FlinkRunner - --parallelism=${parallelism} - --input=${input} - --output=${output} + --parallelism=${flink.examples.parallelism} + --input=${flink.examples.input} + --output=${flink.examples.output} From 4d964734c719037dd6e7e2a3c256da574514758a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 19:51:22 -0800 Subject: [PATCH 235/279] Show timestamps on log lines in Jenkins --- .jenkins/common_job_properties.groovy | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy index f3a8a07952a0..9b1a5fb68724 100644 --- a/.jenkins/common_job_properties.groovy +++ b/.jenkins/common_job_properties.groovy @@ -24,6 +24,7 @@ class common_job_properties { static def setTopLevelJobProperties(def context, def default_branch = 'master', def default_timeout = 100) { + // GitHub project. context.properties { githubProjectUrl('https://github.com/apache/incubator-beam/') @@ -47,7 +48,7 @@ class common_job_properties { remote { url('https://github.com/apache/incubator-beam.git') refspec('+refs/heads/*:refs/remotes/origin/* ' + - '+refs/pull/*:refs/remotes/origin/pr/*') + '+refs/pull/*/head:refs/remotes/origin/pr/*') } branch('${sha1}') extensions { @@ -134,6 +135,8 @@ class common_job_properties { // Sets common config for Maven jobs. static def setMavenConfig(def context) { context.mavenInstallation('Maven 3.3.3') + context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true') + context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS') context.rootPOM('pom.xml') // Use a repository local to the workspace for better isolation of jobs. context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE) From 2148adb824534214bd6c21fc3395abb5c9abdb57 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 19 Dec 2016 11:18:59 -0800 Subject: [PATCH 236/279] Fix seed job fetch spec --- .jenkins/common_job_properties.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy index 9b1a5fb68724..7a886040941c 100644 --- a/.jenkins/common_job_properties.groovy +++ b/.jenkins/common_job_properties.groovy @@ -48,7 +48,7 @@ class common_job_properties { remote { url('https://github.com/apache/incubator-beam.git') refspec('+refs/heads/*:refs/remotes/origin/* ' + - '+refs/pull/*/head:refs/remotes/origin/pr/*') + '+refs/pull/*:refs/remotes/origin/pr/*') } branch('${sha1}') extensions { From 998cabc8bbbf8d08d7bfad71e9376707388f5c5c Mon Sep 17 00:00:00 2001 From: bchambers Date: Thu, 15 Dec 2016 17:04:59 -0800 Subject: [PATCH 237/279] Add RunnableOnService test for Metrics Add UsesMetrics interface and exclude from runners that don't yet support Metrics Add Serializability as needed for Metrics to be created during pipeline construction Remove test from DirectRunnerTest --- runners/apex/pom.xml | 3 +- .../beam/runners/direct/DirectRunnerTest.java | 39 ------------ runners/flink/runner/pom.xml | 6 +- runners/google-cloud-dataflow-java/pom.xml | 3 +- runners/spark/pom.xml | 3 +- .../apache/beam/sdk/metrics/MetricName.java | 3 +- .../org/apache/beam/sdk/metrics/Metrics.java | 5 +- .../apache/beam/sdk/testing/UsesMetrics.java | 24 +++++++ .../beam/sdk/metrics/MetricMatchers.java | 4 +- .../apache/beam/sdk/metrics/MetricsTest.java | 63 ++++++++++++++++++- 10 files changed, 103 insertions(+), 50 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index f71637c66fab..d03964daaa7e 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -186,7 +186,8 @@ org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, - org.apache.beam.sdk.testing.UsesSplittableParDo + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesMetrics none true diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index eb0f344dabe5..eafb788d9c96 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -18,8 +18,6 @@ package org.apache.beam.runners.direct; import static com.google.common.base.Preconditions.checkState; -import static org.apache.beam.sdk.metrics.MetricMatchers.metricResult; -import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isA; @@ -37,7 +35,6 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -48,13 +45,6 @@ import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.Distribution; -import org.apache.beam.sdk.metrics.DistributionResult; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; @@ -467,35 +457,6 @@ public Long decode(InputStream inStream, Context context) throws IOException { } } - @Test - public void testMetrics() throws Exception { - Pipeline pipeline = getPipeline(); - pipeline - .apply(Create.of(5, 8, 13)) - .apply("MyStep", ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - Counter count = Metrics.counter(DirectRunnerTest.class, "count"); - Distribution values = Metrics.distribution(DirectRunnerTest.class, "input"); - - count.inc(); - values.update(c.element()); - } - })); - PipelineResult result = pipeline.run(); - MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder() - .addNameFilter(MetricNameFilter.inNamespace(DirectRunnerTest.class)) - .build()); - - final String stepName = "MyStep/AnonymousParDo/AnonymousParMultiDo"; - assertThat(metrics.counters(), contains( - metricResult(DirectRunnerTest.class.getName(), "count", stepName, 3L, 3L))); - assertThat(metrics.distributions(), contains( - metricResult(DirectRunnerTest.class.getName(), "input", stepName, - DistributionResult.create(26L, 3L, 5L, 13L), - DistributionResult.create(26L, 3L, 5L, 13L)))); - } - private static class MustSplitSource extends BoundedSource{ public static BoundedSource of(BoundedSource underlying) { return new MustSplitSource<>(underlying); diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 09773e106e6e..7f49372aff17 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -56,7 +56,8 @@ org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, - org.apache.beam.sdk.testing.UsesSplittableParDo + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesMetrics none true @@ -86,7 +87,8 @@ org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, - org.apache.beam.sdk.testing.UsesSplittableParDo + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesMetrics none true diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 46ac7ef11c2d..0094791f0d30 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -80,7 +80,8 @@ org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, - org.apache.beam.sdk.testing.UsesSplittableParDo + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesMetrics org.apache.beam.sdk.transforms.FlattenTest diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 5a2fe87ab4cb..309e1ffbef3f 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -75,7 +75,8 @@ org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, - org.apache.beam.sdk.testing.UsesSplittableParDo + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesMetrics 1 false diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java index 843a88575012..3c7704336ed8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricName.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.metrics; import com.google.auto.value.AutoValue; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; @@ -28,7 +29,7 @@ */ @Experimental(Kind.METRICS) @AutoValue -public abstract class MetricName { +public abstract class MetricName implements Serializable { /** The namespace associated with this metric. */ public abstract String namespace(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java index b72a0b244a17..045e076b2a78 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.metrics; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; @@ -58,7 +59,7 @@ public static Distribution distribution(Class namespace, String name) { } /** Implementation of {@link Counter} that delegates to the instance for the current context. */ - private static class DelegatingCounter implements Counter { + private static class DelegatingCounter implements Counter, Serializable { private final MetricName name; private DelegatingCounter(MetricName name) { @@ -92,7 +93,7 @@ private DelegatingCounter(MetricName name) { /** * Implementation of {@link Distribution} that delegates to the instance for the current context. */ - private static class DelegatingDistribution implements Distribution { + private static class DelegatingDistribution implements Distribution, Serializable { private final MetricName name; private DelegatingDistribution(MetricName name) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.java new file mode 100644 index 000000000000..261354c240d3 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMetrics.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; + +/** + * Category tag for validation tests which utilize {@link org.apache.beam.sdk.metrics.Metrics}. + */ +public interface UsesMetrics {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java index 6cd4c5245eca..798d9d41fd30 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricMatchers.java @@ -78,7 +78,7 @@ public static Matcher> metricResult( protected boolean matchesSafely(MetricResult item) { return Objects.equals(namespace, item.name().namespace()) && Objects.equals(name, item.name().name()) - && Objects.equals(step, item.step()) + && item.step().contains(step) && Objects.equals(committed, item.committed()) && Objects.equals(attempted, item.attempted()); } @@ -109,7 +109,7 @@ protected void describeMismatchSafely(MetricResult item, Description mismatch .appendText(" != ").appendValue(item.name().name()); } - if (!Objects.equals(step, item.step())) { + if (!item.step().contains(step)) { mismatchDescription .appendText("step: ").appendValue(step) .appendText(" != ").appendValue(item.step()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java index 732cb346d20a..075df19b168c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java @@ -18,18 +18,30 @@ package org.apache.beam.sdk.metrics; +import static org.apache.beam.sdk.metrics.MetricMatchers.metricResult; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testing.RunnableOnService; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.UsesMetrics; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; import org.hamcrest.CoreMatchers; import org.junit.After; import org.junit.Test; +import org.junit.experimental.categories.Category; /** * Tests for {@link Metrics}. */ -public class MetricsTest { +public class MetricsTest implements Serializable { private static final String NS = "test"; private static final String NAME = "name"; @@ -95,4 +107,53 @@ public void counterToCell() { counter.dec(); assertThat(cell.getCumulative(), CoreMatchers.equalTo(42L)); } + + @Category({RunnableOnService.class, UsesMetrics.class}) + @Test + public void metricsReportToQuery() { + final Counter count = Metrics.counter(MetricsTest.class, "count"); + Pipeline pipeline = TestPipeline.create(); + pipeline + .apply(Create.of(5, 8, 13)) + .apply("MyStep1", ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + Distribution values = Metrics.distribution(MetricsTest.class, "input"); + count.inc(); + values.update(c.element()); + + c.output(c.element()); + c.output(c.element()); + } + })) + .apply("MyStep2", ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + Distribution values = Metrics.distribution(MetricsTest.class, "input"); + count.inc(); + values.update(c.element()); + } + })); + PipelineResult result = pipeline.run(); + + result.waitUntilFinish(); + + MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder() + .addNameFilter(MetricNameFilter.inNamespace(MetricsTest.class)) + .build()); + // TODO: BEAM-1169: Metrics shouldn't verify the physical values tightly. + assertThat(metrics.counters(), hasItem( + metricResult(MetricsTest.class.getName(), "count", "MyStep1", 3L, 3L))); + assertThat(metrics.distributions(), hasItem( + metricResult(MetricsTest.class.getName(), "input", "MyStep1", + DistributionResult.create(26L, 3L, 5L, 13L), + DistributionResult.create(26L, 3L, 5L, 13L)))); + + assertThat(metrics.counters(), hasItem( + metricResult(MetricsTest.class.getName(), "count", "MyStep2", 6L, 6L))); + assertThat(metrics.distributions(), hasItem( + metricResult(MetricsTest.class.getName(), "input", "MyStep2", + DistributionResult.create(52L, 6L, 5L, 13L), + DistributionResult.create(52L, 6L, 5L, 13L)))); + } } From 627ccb522d7d818aebe2256246e81af2d3062d11 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 19 Dec 2016 11:39:29 -0800 Subject: [PATCH 238/279] More escaping in Jenkins timestamp spec --- .jenkins/common_job_properties.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy index 7a886040941c..388023654e22 100644 --- a/.jenkins/common_job_properties.groovy +++ b/.jenkins/common_job_properties.groovy @@ -136,7 +136,7 @@ class common_job_properties { static def setMavenConfig(def context) { context.mavenInstallation('Maven 3.3.3') context.mavenOpts('-Dorg.slf4j.simpleLogger.showDateTime=true') - context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\'T\'HH:mm:ss.SSS') + context.mavenOpts('-Dorg.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd\\\'T\\\'HH:mm:ss.SSS') context.rootPOM('pom.xml') // Use a repository local to the workspace for better isolation of jobs. context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE) From a2f44923c8a8db533ad0bb6f545a96cf5007cfa5 Mon Sep 17 00:00:00 2001 From: Joshua Litt Date: Sat, 17 Dec 2016 11:12:12 -0800 Subject: [PATCH 239/279] Change counter name in TestDataflowRunner --- .../dataflow/testing/TestDataflowRunner.java | 29 +++++++++++++++---- .../testing/TestDataflowRunnerTest.java | 16 +++++++++- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java index 4b0fcf218a75..056444877734 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java @@ -61,7 +61,12 @@ */ public class TestDataflowRunner extends PipelineRunner { private static final String TENTATIVE_COUNTER = "tentative"; - private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark"; + // See https://issues.apache.org/jira/browse/BEAM-1170 + // we need to either fix the API or pipe the DRAINED signal through + @VisibleForTesting + static final String LEGACY_WATERMARK_METRIC_SUFFIX = "windmill-data-watermark"; + @VisibleForTesting + static final String WATERMARK_METRIC_SUFFIX = "DataWatermark"; private static final long MAX_WATERMARK_VALUE = -2L; private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class); @@ -247,6 +252,23 @@ Optional checkForPAssertSuccess(DataflowPipelineJob job, @Nullable JobM return Optional.absent(); } + /** + * Checks wether a metric is a streaming watermark. + * + * @return true if the metric is a watermark. + */ + boolean isWatermark(MetricUpdate metric) { + if (metric.getName() == null || metric.getName().getName() == null) { + return false; // no name -> shouldn't happen, not the watermark + } + if (metric.getScalar() == null) { + return false; // no scalar value -> not the watermark + } + String name = metric.getName().getName(); + return name.endsWith(LEGACY_WATERMARK_METRIC_SUFFIX) + || name.endsWith(WATERMARK_METRIC_SUFFIX); + } + /** * Check watermarks of the streaming job. At least one watermark metric must exist. * @@ -256,10 +278,7 @@ Optional checkForPAssertSuccess(DataflowPipelineJob job, @Nullable JobM boolean atMaxWatermark(DataflowPipelineJob job, JobMetrics metrics) { boolean hasMaxWatermark = false; for (MetricUpdate metric : metrics.getMetrics()) { - if (metric.getName() == null - || metric.getName().getName() == null - || !metric.getName().getName().endsWith(WATERMARK_METRIC_SUFFIX) - || metric.getScalar() == null) { + if (!isWatermark(metric)) { continue; } BigDecimal watermark = (BigDecimal) metric.getScalar(); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java index 366c6a13835c..da5630b487b2 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.testing; +import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.LEGACY_WATERMARK_METRIC_SUFFIX; +import static org.apache.beam.runners.dataflow.testing.TestDataflowRunner.WATERMARK_METRIC_SUFFIX; import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -95,7 +97,6 @@ public class TestDataflowRunnerTest { @Mock private MockLowLevelHttpRequest request; @Mock private GcsUtil mockGcsUtil; - private static final String WATERMARK_METRIC_SUFFIX = "windmill-data-watermark"; private static final BigDecimal DEFAULT_MAX_WATERMARK = new BigDecimal(-2); private TestDataflowPipelineOptions options; @@ -410,6 +411,19 @@ public void testCheckMaxWatermarkWithSingleWatermarkAtMax() throws IOException { assertTrue(runner.atMaxWatermark(job, metrics)); } + @Test + public void testCheckMaxWatermarkWithLegacyWatermarkAtMax() throws IOException { + DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); + Pipeline p = TestPipeline.create(options); + p.apply(Create.of(1, 2, 3)); + + TestDataflowRunner runner = (TestDataflowRunner) p.getRunner(); + JobMetrics metrics = buildJobMetrics(generateMockStreamingMetrics( + ImmutableMap.of(LEGACY_WATERMARK_METRIC_SUFFIX, DEFAULT_MAX_WATERMARK))); + doReturn(State.RUNNING).when(job).getState(); + assertTrue(runner.atMaxWatermark(job, metrics)); + } + @Test public void testCheckMaxWatermarkWithSingleWatermarkNotAtMax() throws IOException { DataflowPipelineJob job = spy(new DataflowPipelineJob("test-job", options, null)); From 467f7d17c4c96bc57b0160c2d4768ceb303bc561 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 7 Dec 2016 17:35:23 -0800 Subject: [PATCH 240/279] [BEAM-59] initial interfaces and classes of Beam FileSystem. --- .../org/apache/beam/sdk/io/FileSystem.java | 29 ++++ .../beam/sdk/io/FileSystemRegistrar.java | 49 ++++++ .../org/apache/beam/sdk/io/FileSystems.java | 155 ++++++++++++++++++ .../apache/beam/sdk/io/LocalFileSystem.java | 27 +++ .../beam/sdk/io/LocalFileSystemRegistrar.java | 41 +++++ .../apache/beam/sdk/io/FileSystemsTest.java | 104 ++++++++++++ .../sdk/io/LocalFileSystemRegistrarTest.java | 44 +++++ sdks/java/io/google-cloud-platform/pom.xml | 6 + .../sdk/io/gcp/storage/GcsFileSystem.java | 34 ++++ .../gcp/storage/GcsFileSystemRegistrar.java | 42 +++++ .../beam/sdk/io/gcp/storage/package-info.java | 21 +++ .../storage/GcsFileSystemRegistrarTest.java | 51 ++++++ sdks/java/io/hdfs/pom.xml | 6 + .../beam/sdk/io/hdfs/HadoopFileSystem.java | 29 ++++ .../io/hdfs/HadoopFileSystemRegistrar.java | 42 +++++ .../hdfs/HadoopFileSystemRegistrarTest.java | 52 ++++++ 16 files changed, 732 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemRegistrarTest.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java create mode 100644 sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java create mode 100644 sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java create mode 100644 sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java new file mode 100644 index 000000000000..d99040346a95 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystem.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +/** + * File system interface in Beam. + * + *

      It defines APIs for writing file systems agnostic code. + * + *

      All methods are protected, and they are for file system providers to implement. + * Clients should use {@link FileSystems} utility. + */ +public abstract class FileSystem { +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java new file mode 100644 index 000000000000..1d81c1e36428 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystemRegistrar.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +import com.google.auto.service.AutoService; +import java.util.ServiceLoader; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * A registrar that creates {@link FileSystem} instances from {@link PipelineOptions}. + * + *

      {@link FileSystem} creators have the ability to provide a registrar by creating + * a {@link ServiceLoader} entry and a concrete implementation of this interface. + * + *

      It is optional but recommended to use one of the many build time tools such as + * {@link AutoService} to generate the necessary META-INF files automatically. + */ +public interface FileSystemRegistrar { + /** + * Create a {@link FileSystem} from the given {@link PipelineOptions}. + */ + FileSystem fromOptions(@Nullable PipelineOptions options); + + /** + * Get the URI scheme which defines the namespace of the {@link FileSystemRegistrar}. + * + *

      The scheme is required to be unique among all + * {@link FileSystemRegistrar FileSystemRegistrars}. + * + * @see RFC 2396 + */ + String getScheme(); +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java new file mode 100644 index 000000000000..d086ec62a7a7 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.collect.TreeMultimap; +import java.net.URI; +import java.util.Collection; +import java.util.Map; +import java.util.Map.Entry; +import java.util.ServiceLoader; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Pattern; +import javax.annotation.Nonnull; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.common.ReflectHelpers; + +/** + * Clients facing {@link FileSystem} utility. + */ +public class FileSystems { + + public static final String DEFAULT_SCHEME = "default"; + + private static final Pattern URI_SCHEME_PATTERN = Pattern.compile("^[a-zA-Z][-a-zA-Z0-9+.]*$"); + + private static final Map SCHEME_TO_REGISTRAR = + new ConcurrentHashMap<>(); + + private static final Map SCHEME_TO_DEFAULT_CONFIG = + new ConcurrentHashMap<>(); + + static { + loadFileSystemRegistrars(); + } + + /** + * Loads available {@link FileSystemRegistrar} services. + */ + private static void loadFileSystemRegistrars() { + SCHEME_TO_REGISTRAR.clear(); + Set registrars = + Sets.newTreeSet(ReflectHelpers.ObjectsClassComparator.INSTANCE); + registrars.addAll(Lists.newArrayList( + ServiceLoader.load(FileSystemRegistrar.class, ReflectHelpers.findClassLoader()))); + + verifySchemesAreUnique(registrars); + + for (FileSystemRegistrar registrar : registrars) { + SCHEME_TO_REGISTRAR.put(registrar.getScheme().toLowerCase(), registrar); + } + } + + /** + * Sets the default configuration to be used with a {@link FileSystemRegistrar} for the provided + * {@code scheme}. + * + *

      Syntax:

      scheme = alpha *( alpha | digit | "+" | "-" | "." )
      + * Upper case letters are treated as the same as lower case letters. + */ + public static void setDefaultConfig(String scheme, PipelineOptions options) { + String lowerCaseScheme = checkNotNull(scheme, "scheme").toLowerCase(); + checkArgument( + URI_SCHEME_PATTERN.matcher(lowerCaseScheme).matches(), + String.format("Scheme: [%s] doesn't match URI syntax: %s", + lowerCaseScheme, URI_SCHEME_PATTERN.pattern())); + checkArgument( + SCHEME_TO_REGISTRAR.containsKey(lowerCaseScheme), + String.format("No FileSystemRegistrar found for scheme: [%s].", lowerCaseScheme)); + SCHEME_TO_DEFAULT_CONFIG.put(lowerCaseScheme, checkNotNull(options, "options")); + } + + @VisibleForTesting + static PipelineOptions getDefaultConfig(String scheme) { + return SCHEME_TO_DEFAULT_CONFIG.get(scheme.toLowerCase()); + } + + /** + * Internal method to get {@link FileSystem} for {@code spec}. + */ + @VisibleForTesting + static FileSystem getFileSystemInternal(URI uri) { + String lowerCaseScheme = (uri.getScheme() != null + ? uri.getScheme().toLowerCase() : LocalFileSystemRegistrar.LOCAL_FILE_SCHEME); + return getRegistrarInternal(lowerCaseScheme).fromOptions(getDefaultConfig(lowerCaseScheme)); + } + + /** + * Internal method to get {@link FileSystemRegistrar} for {@code scheme}. + */ + @VisibleForTesting + static FileSystemRegistrar getRegistrarInternal(String scheme) { + String lowerCaseScheme = scheme.toLowerCase(); + if (SCHEME_TO_REGISTRAR.containsKey(lowerCaseScheme)) { + return SCHEME_TO_REGISTRAR.get(lowerCaseScheme); + } else if (SCHEME_TO_REGISTRAR.containsKey(DEFAULT_SCHEME)) { + return SCHEME_TO_REGISTRAR.get(DEFAULT_SCHEME); + } else { + throw new IllegalStateException("Unable to find registrar for " + scheme); + } + } + + @VisibleForTesting + static void verifySchemesAreUnique(Set registrars) { + Multimap registrarsBySchemes = + TreeMultimap.create(Ordering.natural(), Ordering.arbitrary()); + + for (FileSystemRegistrar registrar : registrars) { + registrarsBySchemes.put(registrar.getScheme().toLowerCase(), registrar); + } + for (Entry> entry + : registrarsBySchemes.asMap().entrySet()) { + if (entry.getValue().size() > 1) { + String conflictingRegistrars = Joiner.on(", ").join( + FluentIterable.from(entry.getValue()) + .transform(new Function() { + @Override + public String apply(@Nonnull FileSystemRegistrar input) { + return input.getClass().getName(); + }}) + .toSortedList(Ordering.natural())); + throw new IllegalStateException(String.format( + "Scheme: [%s] has conflicting registrars: [%s]", + entry.getKey(), + conflictingRegistrars)); + } + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java new file mode 100644 index 000000000000..23c2a920431f --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +/** + * {@link FileSystem} implementation for local files. + */ +class LocalFileSystem extends FileSystem { + + LocalFileSystem() { + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java new file mode 100644 index 000000000000..75a38e8c922e --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystemRegistrar.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +import com.google.auto.service.AutoService; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * {@link AutoService} registrar for the {@link FileSystem}. + */ +@AutoService(FileSystemRegistrar.class) +public class LocalFileSystemRegistrar implements FileSystemRegistrar { + + static final String LOCAL_FILE_SCHEME = "file"; + + @Override + public FileSystem fromOptions(@Nullable PipelineOptions options) { + return new LocalFileSystem(); + } + + @Override + public String getScheme() { + return LOCAL_FILE_SCHEME; + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java new file mode 100644 index 000000000000..9b41b9831407 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +import com.google.common.collect.Sets; +import java.net.URI; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link FileSystems}. + */ +@RunWith(JUnit4.class) +public class FileSystemsTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testSetDefaultConfig() throws Exception { + PipelineOptions first = PipelineOptionsFactory.create(); + PipelineOptions second = PipelineOptionsFactory.create(); + FileSystems.setDefaultConfig("file", first); + assertEquals(first, FileSystems.getDefaultConfig("file")); + assertEquals(first, FileSystems.getDefaultConfig("FILE")); + + FileSystems.setDefaultConfig("FILE", second); + assertNotEquals(first, FileSystems.getDefaultConfig("file")); + assertNotEquals(first, FileSystems.getDefaultConfig("FILE")); + assertEquals(second, FileSystems.getDefaultConfig("file")); + assertEquals(second, FileSystems.getDefaultConfig("FILE")); + } + + @Test + public void testSetDefaultConfigNotFound() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("No FileSystemRegistrar found for scheme: [gs-s3]."); + FileSystems.setDefaultConfig("gs-s3", PipelineOptionsFactory.create()); + } + + @Test + public void testSetDefaultConfigInvalidScheme() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Scheme: [gs:] doesn't match URI syntax"); + FileSystems.setDefaultConfig("gs:", PipelineOptionsFactory.create()); + } + + @Test + public void testGetLocalFileSystem() throws Exception { + assertTrue( + FileSystems.getFileSystemInternal(URI.create("~/home/")) instanceof LocalFileSystem); + assertTrue( + FileSystems.getFileSystemInternal(URI.create("file://home")) instanceof LocalFileSystem); + assertTrue( + FileSystems.getFileSystemInternal(URI.create("FILE://home")) instanceof LocalFileSystem); + assertTrue( + FileSystems.getFileSystemInternal(URI.create("File://home")) instanceof LocalFileSystem); + } + + @Test + public void testVerifySchemesAreUnique() throws Exception { + thrown.expect(RuntimeException.class); + thrown.expectMessage("Scheme: [file] has conflicting registrars"); + FileSystems.verifySchemesAreUnique( + Sets.newHashSet( + new LocalFileSystemRegistrar(), + new FileSystemRegistrar() { + @Override + public FileSystem fromOptions(@Nullable PipelineOptions options) { + return null; + } + + @Override + public String getScheme() { + return "FILE"; + } + })); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemRegistrarTest.java new file mode 100644 index 000000000000..e4e8326ea8cf --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemRegistrarTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io; + +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; +import java.util.ServiceLoader; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link LocalFileSystemRegistrar}. + */ +@RunWith(JUnit4.class) +public class LocalFileSystemRegistrarTest { + + @Test + public void testServiceLoader() { + for (FileSystemRegistrar registrar + : Lists.newArrayList(ServiceLoader.load(FileSystemRegistrar.class).iterator())) { + if (registrar instanceof LocalFileSystemRegistrar) { + return; + } + } + fail("Expected to find " + LocalFileSystemRegistrar.class); + } +} diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index d3b5fed573fb..76bdc45f1ce8 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -99,6 +99,12 @@ google-api-services-bigquery + + com.google.auto.service + auto-service + true + + com.google.cloud.bigdataoss util diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java new file mode 100644 index 000000000000..4b03e2728753 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystem.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.storage; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.io.FileSystem; +import org.apache.beam.sdk.options.GcsOptions; + +/** + * {@link FileSystem} implementation for Google Cloud Storage. + */ +class GcsFileSystem extends FileSystem { + private final GcsOptions options; + + GcsFileSystem(GcsOptions options) { + this.options = checkNotNull(options, "options"); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java new file mode 100644 index 000000000000..10452a156d45 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrar.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.storage; + +import com.google.auto.service.AutoService; +import javax.annotation.Nonnull; +import org.apache.beam.sdk.io.FileSystem; +import org.apache.beam.sdk.io.FileSystemRegistrar; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * {@link AutoService} registrar for the {@link GcsFileSystem}. + */ +@AutoService(FileSystemRegistrar.class) +public class GcsFileSystemRegistrar implements FileSystemRegistrar { + + @Override + public FileSystem fromOptions(@Nonnull PipelineOptions options) { + return new GcsFileSystem(options.as(GcsOptions.class)); + } + + @Override + public String getScheme() { + return "gs"; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java new file mode 100644 index 000000000000..b5378be90825 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/storage/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ +/** + * Defines IO connectors for Google Cloud Storage. + */ +package org.apache.beam.sdk.io.gcp.storage; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.java new file mode 100644 index 000000000000..ecac8f64cdc0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsFileSystemRegistrarTest.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.sdk.io.gcp.storage; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; +import java.util.ServiceLoader; + +import org.apache.beam.sdk.io.FileSystemRegistrar; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link GcsFileSystemRegistrar}. + */ +@RunWith(JUnit4.class) +public class GcsFileSystemRegistrarTest { + + @Test + public void testServiceLoader() { + for (FileSystemRegistrar registrar + : Lists.newArrayList(ServiceLoader.load(FileSystemRegistrar.class).iterator())) { + if (registrar instanceof GcsFileSystemRegistrar) { + assertEquals("gs", registrar.getScheme()); + assertTrue(registrar.fromOptions(PipelineOptionsFactory.create()) instanceof GcsFileSystem); + return; + } + } + fail("Expected to find " + GcsFileSystemRegistrar.class); + } +} diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 772276bf3113..b171cfe08397 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -63,6 +63,12 @@ jackson-annotations + + com.google.auto.service + auto-service + true + + com.google.guava guava diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java new file mode 100644 index 000000000000..b94a089b3a2a --- /dev/null +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.hdfs; + +import org.apache.beam.sdk.io.FileSystem; + +/** + * Adapts {@link org.apache.hadoop.fs.FileSystem} connectors to be used as + * Apache Beam {@link FileSystem FileSystems}. + */ +class HadoopFileSystem extends FileSystem { + + HadoopFileSystem() {} +} diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java new file mode 100644 index 000000000000..1471cb0dbdc2 --- /dev/null +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.hdfs; + +import com.google.auto.service.AutoService; +import javax.annotation.Nonnull; +import org.apache.beam.sdk.io.FileSystem; +import org.apache.beam.sdk.io.FileSystemRegistrar; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * {@link AutoService} registrar for the {@link HadoopFileSystem}. + */ +@AutoService(FileSystemRegistrar.class) +public class HadoopFileSystemRegistrar implements FileSystemRegistrar { + + @Override + public FileSystem fromOptions(@Nonnull PipelineOptions options) { + return new HadoopFileSystem(); + } + + @Override + public String getScheme() { + return FileSystems.DEFAULT_SCHEME; + } +} diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java new file mode 100644 index 000000000000..22a439acdb8c --- /dev/null +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.hdfs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; +import java.util.ServiceLoader; +import org.apache.beam.sdk.io.FileSystemRegistrar; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link HadoopFileSystemRegistrar}. + */ +@RunWith(JUnit4.class) +public class HadoopFileSystemRegistrarTest { + + @Test + public void testServiceLoader() { + for (FileSystemRegistrar registrar + : Lists.newArrayList(ServiceLoader.load(FileSystemRegistrar.class).iterator())) { + if (registrar instanceof HadoopFileSystemRegistrar) { + assertEquals(FileSystems.DEFAULT_SCHEME, registrar.getScheme()); + assertTrue( + registrar.fromOptions(PipelineOptionsFactory.create()) instanceof HadoopFileSystem); + return; + } + } + fail("Expected to find " + HadoopFileSystemRegistrar.class); + } +} From 85422f999752fc827113609be0ba72cc64a9d3b3 Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Mon, 19 Dec 2016 11:13:49 -0800 Subject: [PATCH 241/279] Disable automatic archiving of Maven builds From the Web UI: > If checked, Jenkins will not automatically archive all artifacts generated by this project. If you wish to archive the results of this build within Jenkins, you will need to use the "Archive the artifacts" post-build action below. --- .jenkins/common_job_properties.groovy | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy index 388023654e22..e1688ec060cf 100644 --- a/.jenkins/common_job_properties.groovy +++ b/.jenkins/common_job_properties.groovy @@ -140,6 +140,10 @@ class common_job_properties { context.rootPOM('pom.xml') // Use a repository local to the workspace for better isolation of jobs. context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE) + // Disable archiving the built artifacts by default, as this is slow and flaky. + // We can usually recreate them easily, and we can also opt-in individual jobs + // to artifact archiving. + context.archivingDisabled(true) } // Sets common config for PreCommit jobs. From 83068990d803edabee7aefa32de9542f84d087ac Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Sun, 18 Dec 2016 18:25:33 +0200 Subject: [PATCH 242/279] Migrated the beam-examples-java module to TestPipeline as a JUnit rule. --- .../java/org/apache/beam/examples/WordCountTest.java | 7 ++++--- .../beam/examples/complete/AutoCompleteTest.java | 11 ++++------- .../org/apache/beam/examples/complete/TfIdfTest.java | 6 ++++-- .../examples/complete/TopWikipediaSessionsTest.java | 7 +++++-- .../beam/examples/cookbook/DistinctExampleTest.java | 9 ++++----- .../beam/examples/cookbook/JoinExamplesTest.java | 6 ++++-- .../beam/examples/cookbook/TriggerExampleTest.java | 6 ++++-- 7 files changed, 29 insertions(+), 23 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java index c8809de0b888..0f8e4dcebdfd 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountTest.java @@ -22,7 +22,6 @@ import org.apache.beam.examples.WordCount.CountWords; import org.apache.beam.examples.WordCount.ExtractWordsFn; import org.apache.beam.examples.WordCount.FormatAsTextFn; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -34,6 +33,7 @@ import org.apache.beam.sdk.values.PCollection; import org.hamcrest.CoreMatchers; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -68,12 +68,13 @@ public void testExtractWordsFn() throws Exception { static final String[] COUNTS_ARRAY = new String[] { "hi: 5", "there: 1", "sue: 2", "bob: 2"}; + @Rule + public TestPipeline p = TestPipeline.create(); + /** Example test that tests a PTransform by using an in-memory input and inspecting the output. */ @Test @Category(RunnableOnService.class) public void testCountWords() throws Exception { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of())); PCollection output = input.apply(new CountWords()) diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java index d7d4dc6c37d9..ef57da48c3e6 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java @@ -24,7 +24,6 @@ import java.util.List; import org.apache.beam.examples.complete.AutoComplete.CompletionCandidate; import org.apache.beam.examples.complete.AutoComplete.ComputeTopCompletions; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -40,6 +39,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -51,6 +51,9 @@ public class AutoCompleteTest implements Serializable { private boolean recursive; + @Rule + public transient TestPipeline p = TestPipeline.create(); + public AutoCompleteTest(Boolean recursive) { this.recursive = recursive; } @@ -77,8 +80,6 @@ public void testAutoComplete() { "blueberry", "cherry"); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(words)); PCollection>> output = @@ -106,8 +107,6 @@ public Boolean apply(KV> element) { public void testTinyAutoComplete() { List words = Arrays.asList("x", "x", "x", "xy", "xy", "xyz"); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(words)); PCollection>> output = @@ -129,8 +128,6 @@ public void testWindowedAutoComplete() { TimestampedValue.of("xB", new Instant(2)), TimestampedValue.of("xB", new Instant(2))); - Pipeline p = TestPipeline.create(); - PCollection input = p .apply(Create.of(words)) .apply(new ReifyTimestamps()); diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java index fded4c02b2a8..03825323871c 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TfIdfTest.java @@ -19,7 +19,6 @@ import java.net.URI; import java.util.Arrays; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringDelegateCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -29,6 +28,7 @@ import org.apache.beam.sdk.transforms.Keys; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -40,11 +40,13 @@ @RunWith(JUnit4.class) public class TfIdfTest { + @Rule + public TestPipeline pipeline = TestPipeline.create(); + /** Test that the example runs. */ @Test @Category(RunnableOnService.class) public void testTfIdf() throws Exception { - Pipeline pipeline = TestPipeline.create(); pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java index 154ea73f70c6..2c50cf24f6ed 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TopWikipediaSessionsTest.java @@ -19,12 +19,12 @@ import com.google.api.services.bigquery.model.TableRow; import java.util.Arrays; -import org.apache.beam.sdk.Pipeline; 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.transforms.Create; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -33,10 +33,13 @@ /** Unit tests for {@link TopWikipediaSessions}. */ @RunWith(JUnit4.class) public class TopWikipediaSessionsTest { + + @Rule + public TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testComputeTopUsers() { - Pipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of(Arrays.asList( diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DistinctExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DistinctExampleTest.java index 6fa66f18c701..6fadbe50bb67 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DistinctExampleTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DistinctExampleTest.java @@ -19,7 +19,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -27,6 +26,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Distinct; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -36,6 +36,9 @@ @RunWith(JUnit4.class) public class DistinctExampleTest { + @Rule + public TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testDistinct() { @@ -48,8 +51,6 @@ public void testDistinct() { "k2", "k3"); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(strings) .withCoder(StringUtf8Coder.of())); @@ -67,8 +68,6 @@ public void testDistinct() { public void testDistinctEmpty() { List strings = Arrays.asList(); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(strings) .withCoder(StringUtf8Coder.of())); diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java index 6c54aff552f5..43e6d013695c 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/JoinExamplesTest.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.beam.examples.cookbook.JoinExamples.ExtractCountryInfoFn; import org.apache.beam.examples.cookbook.JoinExamples.ExtractEventDataFn; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; @@ -32,6 +31,7 @@ import org.apache.beam.sdk.values.PCollection; import org.hamcrest.CoreMatchers; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -80,6 +80,9 @@ public class JoinExamplesTest { + "url: http://cnn.com" }; + @Rule + public TestPipeline p = TestPipeline.create(); + @Test public void testExtractEventDataFn() throws Exception { DoFnTester> extractEventDataFn = @@ -102,7 +105,6 @@ public void testExtractCountryInfoFn() throws Exception { @Test @Category(RunnableOnService.class) public void testJoin() throws java.lang.Exception { - Pipeline p = TestPipeline.create(); PCollection input1 = p.apply("CreateEvent", Create.of(EVENT_ARRAY)); PCollection input2 = p.apply("CreateCC", Create.of(CC_ARRAY)); diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java index bdda22cfb681..ec0b9d4e38b6 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java @@ -26,7 +26,6 @@ import java.util.Map; import org.apache.beam.examples.cookbook.TriggerExample.ExtractFlowInfo; import org.apache.beam.examples.cookbook.TriggerExample.TotalFlow; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; @@ -42,6 +41,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -93,6 +93,9 @@ public class TriggerExampleTest { .set("timing", "ON_TIME") .set("window", "[1970-01-01T00:00:00.000Z..1970-01-01T00:01:00.000Z)"); + @Rule + public TestPipeline pipeline = TestPipeline.create(); + @Test public void testExtractTotalFlow() throws Exception { DoFnTester> extractFlowInfow = DoFnTester @@ -110,7 +113,6 @@ public void testExtractTotalFlow() throws Exception { @Test @Category(RunnableOnService.class) public void testTotalFlow () { - Pipeline pipeline = TestPipeline.create(); PCollection> flow = pipeline .apply(Create.timestamped(TIME_STAMPED_INPUT)) .apply(ParDo.of(new ExtractFlowInfo())); From 7106e8806f32089eab36e6feee4928026bf52714 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Sun, 18 Dec 2016 18:38:11 +0200 Subject: [PATCH 243/279] Migrated the beam-examples-java8 module to TestPipeline as a JUnit rule. --- .../beam/examples/MinimalWordCountJava8Test.java | 6 ++++-- .../beam/examples/complete/game/GameStatsTest.java | 7 ++++--- .../examples/complete/game/HourlyTeamScoreTest.java | 5 +++-- .../beam/examples/complete/game/LeaderBoardTest.java | 11 +++++------ .../beam/examples/complete/game/UserScoreTest.java | 10 +++++----- 5 files changed, 21 insertions(+), 18 deletions(-) diff --git a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java index f3733436746d..c2f3efe92d3a 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/MinimalWordCountJava8Test.java @@ -26,7 +26,6 @@ import java.nio.file.StandardOpenOption; import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.testing.TestPipeline; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptors; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -52,12 +52,14 @@ @RunWith(JUnit4.class) public class MinimalWordCountJava8Test implements Serializable { + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + /** * A basic smoke test that ensures there is no crash at pipeline construction time. */ @Test public void testMinimalWordCountJava8() throws Exception { - Pipeline p = TestPipeline.create(); p.getOptions().as(GcsOptions.class).setGcsUtil(buildMockGcsUtil()); p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java index 8e12c46f4a4e..da2bb91a7ad2 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/GameStatsTest.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.examples.complete.game.GameStats.CalculateSpammyUsers; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -29,6 +28,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -58,12 +58,13 @@ public class GameStatsTest implements Serializable { static final List> SPAMMERS = Arrays.asList( KV.of("Robot-2", 66), KV.of("Robot-1", 116)); + @Rule + public TestPipeline p = TestPipeline.create(); + /** Test the calculation of 'spammy users'. */ @Test @Category(RunnableOnService.class) public void testCalculateSpammyUsers() throws Exception { - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(USER_SCORES)); PCollection> output = input.apply(new CalculateSpammyUsers()); diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java index cc42d5219591..34a0744fb187 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; import org.apache.beam.examples.complete.game.UserScore.ParseEventFn; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -81,12 +81,13 @@ public class HourlyTeamScoreTest implements Serializable { KV.of("user18_BananaEmu", 1), KV.of("user18_ApricotCaneToad", 14) }; + @Rule + public TestPipeline p = TestPipeline.create(); /** Test the filtering. */ @Test @Category(RunnableOnService.class) public void testUserScoresFilter() throws Exception { - Pipeline p = TestPipeline.create(); final Instant startMinTimestamp = new Instant(1447965680000L); diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java index 2b51da1fcec2..745c210c5635 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -54,6 +55,8 @@ public class LeaderBoardTest implements Serializable { private static final Duration TEAM_WINDOW_DURATION = Duration.standardMinutes(20); private Instant baseTime = new Instant(0); + @Rule + public TestPipeline p = TestPipeline.create(); /** * Some example users, on two separate teams. */ @@ -84,7 +87,6 @@ public String getTeam() { */ @Test public void testTeamScoresOnTime() { - TestPipeline p = TestPipeline.create(); TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) // Start at the epoch @@ -120,7 +122,6 @@ public void testTeamScoresOnTime() { */ @Test public void testTeamScoresSpeculative() { - TestPipeline p = TestPipeline.create(); TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) // Start at the epoch @@ -170,7 +171,6 @@ public void testTeamScoresSpeculative() { */ @Test public void testTeamScoresUnobservablyLate() { - TestPipeline p = TestPipeline.create(); BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) @@ -208,7 +208,6 @@ public void testTeamScoresUnobservablyLate() { */ @Test public void testTeamScoresObservablyLate() { - TestPipeline p = TestPipeline.create(); Instant firstWindowCloses = baseTime.plus(ALLOWED_LATENESS).plus(TEAM_WINDOW_DURATION); TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) @@ -268,7 +267,6 @@ public void testTeamScoresObservablyLate() { */ @Test public void testTeamScoresDroppablyLate() { - TestPipeline p = TestPipeline.create(); BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); TestStream infos = TestStream.create(AvroCoder.of(GameActionInfo.class)) @@ -299,6 +297,8 @@ public void testTeamScoresDroppablyLate() { // No elements are added before the watermark passes the end of the window plus the allowed // lateness, so no refinement should be emitted PAssert.that(teamScores).inFinalPane(window).empty(); + + p.run().waitUntilFinish(); } /** @@ -308,7 +308,6 @@ public void testTeamScoresDroppablyLate() { */ @Test public void testUserScore() { - TestPipeline p = TestPipeline.create(); TestStream infos = TestStream.create(AvroCoder.of(GameActionInfo.class)) diff --git a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java index 39de33326861..3b77b2649622 100644 --- a/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/UserScoreTest.java @@ -23,7 +23,6 @@ import org.apache.beam.examples.complete.game.UserScore.ExtractAndSumScore; import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; import org.apache.beam.examples.complete.game.UserScore.ParseEventFn; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -81,7 +81,10 @@ public class UserScoreTest implements Serializable { KV.of("AndroidGreenKookaburra", 23), KV.of("BisqueBilby", 14)); - /** Test the {@link ParseEventFn} {@link DoFn}. */ + @Rule + public TestPipeline p = TestPipeline.create(); + + /** Test the {@link ParseEventFn} {@link org.apache.beam.sdk.transforms.DoFn}. */ @Test public void testParseEventFn() throws Exception { DoFnTester parseEventFn = @@ -98,7 +101,6 @@ public void testParseEventFn() throws Exception { @Test @Category(RunnableOnService.class) public void testUserScoreSums() throws Exception { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of())); @@ -117,7 +119,6 @@ public void testUserScoreSums() throws Exception { @Test @Category(RunnableOnService.class) public void testTeamScoreSums() throws Exception { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of())); @@ -136,7 +137,6 @@ public void testTeamScoreSums() throws Exception { @Test @Category(RunnableOnService.class) public void testUserScoresBadInput() throws Exception { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(GAME_EVENTS2).withCoder(StringUtf8Coder.of())); From b6710251d8bb5d1968aea2258ce5878b43368dd5 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Sun, 18 Dec 2016 18:51:31 +0200 Subject: [PATCH 244/279] Migrated the beam-runners-core module to TestPipeline as a JUnit rule. --- .../runners/core/PushbackSideInputDoFnRunnerTest.java | 5 ++++- .../apache/beam/runners/core/SplittableParDoTest.java | 10 ++++++++-- .../core/UnboundedReadFromBoundedSourceTest.java | 5 +++-- 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java index a1cdbf6dce04..251c7c2fc3a4 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java @@ -47,6 +47,7 @@ import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -63,10 +64,12 @@ public class PushbackSideInputDoFnRunnerTest { private TestDoFnRunner underlying; private PCollectionView singletonView; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { MockitoAnnotations.initMocks(this); - TestPipeline p = TestPipeline.create(); PCollection created = p.apply(Create.of(1, 2, 3)); singletonView = created diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index cf96b660bea6..0f0b106d2ebd 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -57,6 +57,7 @@ import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -132,9 +133,13 @@ private ParDo.BoundMulti makeParDo(DoFn fn) { return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty()); } + @Rule + public TestPipeline pipeline = TestPipeline.create(); + @Test public void testBoundednessForBoundedFn() { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + DoFn boundedFn = new BoundedFakeFn(); assertEquals( "Applying a bounded SDF to a bounded collection produces a bounded collection", @@ -154,7 +159,8 @@ public void testBoundednessForBoundedFn() { @Test public void testBoundednessForUnboundedFn() { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + DoFn unboundedFn = new UnboundedFakeFn(); assertEquals( "Applying an unbounded SDF to a bounded collection produces a bounded collection", diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java index 7fd8807585d1..86450f275e9c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java @@ -78,6 +78,9 @@ public class UnboundedReadFromBoundedSourceTest { @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Rule + public TestPipeline p = TestPipeline.create(); + @Test public void testCheckpointCoderNulls() throws Exception { CheckpointCoder coder = new CheckpointCoder<>(StringUtf8Coder.of()); @@ -97,8 +100,6 @@ public void testBoundedToUnboundedSourceAdapter() throws Exception { UnboundedSource> unboundedSource = new BoundedToUnboundedSourceAdapter<>(boundedSource); - Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Read.from(unboundedSource).withMaxNumRecords(numElements)); From 09c404a6c407898fcbc2fd22797cba4da8839b93 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Mon, 19 Dec 2016 10:20:16 +0200 Subject: [PATCH 245/279] Migrated the beam-runners-direct-java module to TestPipeline as a JUnit rule. --- .../direct/BoundedReadEvaluatorFactoryTest.java | 13 ++++++------- .../runners/direct/CloningBundleFactoryTest.java | 8 ++------ .../beam/runners/direct/CommittedResultTest.java | 6 +++++- .../CopyOnAccessInMemoryStateInternalsTest.java | 11 +++++++++-- .../runners/direct/DirectGraphVisitorTest.java | 3 ++- .../beam/runners/direct/EvaluationContextTest.java | 7 ++++--- .../direct/FlattenEvaluatorFactoryTest.java | 6 ++++-- .../direct/GroupByKeyEvaluatorFactoryTest.java | 5 ++++- .../direct/GroupByKeyOnlyEvaluatorFactoryTest.java | 5 ++++- .../ImmutabilityCheckingBundleFactoryTest.java | 4 +++- .../direct/ImmutabilityEnforcementFactoryTest.java | 3 ++- .../direct/ImmutableListBundleFactoryTest.java | 14 +++++++++++--- .../direct/KeyedPValueTrackingVisitorTest.java | 6 +++--- .../beam/runners/direct/ParDoEvaluatorTest.java | 5 ++++- .../runners/direct/SideInputContainerTest.java | 5 +++-- .../direct/StatefulParDoEvaluatorFactoryTest.java | 7 +++++-- .../runners/direct/StepTransformResultTest.java | 5 ++++- .../direct/TestStreamEvaluatorFactoryTest.java | 5 ++++- .../beam/runners/direct/TransformExecutorTest.java | 4 +++- .../direct/UnboundedReadEvaluatorFactoryTest.java | 9 ++++----- .../runners/direct/ViewEvaluatorFactoryTest.java | 5 ++++- .../direct/WatermarkCallbackExecutorTest.java | 5 ++++- .../beam/runners/direct/WatermarkManagerTest.java | 6 ++++-- .../runners/direct/WindowEvaluatorFactoryTest.java | 5 ++++- .../direct/WriteWithShardingFactoryTest.java | 14 +++++++------- 25 files changed, 109 insertions(+), 57 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index acb14441b407..97eae27029d9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -62,6 +62,7 @@ import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -82,11 +83,13 @@ public class BoundedReadEvaluatorFactoryTest { private BundleFactory bundleFactory; private AppliedPTransform longsProducer; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { MockitoAnnotations.initMocks(this); source = CountingSource.upTo(10L); - TestPipeline p = TestPipeline.create(); longs = p.apply(Read.from(source)); factory = @@ -142,7 +145,7 @@ public void boundedSourceEvaluatorProducesDynamicSplits() throws Exception { elems[i] = (long) i; } PCollection read = - TestPipeline.create().apply(Read.from(new TestSource<>(VarLongCoder.of(), 5, elems))); + p.apply(Read.from(new TestSource<>(VarLongCoder.of(), 5, elems))); AppliedPTransform transform = DirectGraphs.getProducer(read); Collection> unreadInputs = new BoundedReadEvaluatorFactory.InputProvider(context).getInitialInputs(transform, 1); @@ -191,8 +194,7 @@ public void boundedSourceEvaluatorDynamicSplitsUnsplittable() throws Exception { BoundedReadEvaluatorFactory factory = new BoundedReadEvaluatorFactory(context, 0L); PCollection read = - TestPipeline.create() - .apply(Read.from(SourceTestUtils.toUnsplittableSource(CountingSource.upTo(10L)))); + p.apply(Read.from(SourceTestUtils.toUnsplittableSource(CountingSource.upTo(10L)))); AppliedPTransform transform = DirectGraphs.getProducer(read); when(context.createRootBundle()).thenReturn(bundleFactory.createRootBundle()); @@ -298,8 +300,6 @@ public void boundedSourceInMemoryTransformEvaluatorShardsOfSource() throws Excep @Test public void boundedSourceEvaluatorClosesReader() throws Exception { TestSource source = new TestSource<>(BigEndianLongCoder.of(), 1L, 2L, 3L); - - TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); AppliedPTransform sourceTransform = DirectGraphs.getProducer(pcollection); @@ -320,7 +320,6 @@ public void boundedSourceEvaluatorClosesReader() throws Exception { public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception { TestSource source = new TestSource<>(BigEndianLongCoder.of()); - TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); AppliedPTransform sourceTransform = DirectGraphs.getProducer(pcollection); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java index bafab59b73d5..e5299a2b3b85 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java @@ -62,6 +62,8 @@ @RunWith(JUnit4.class) public class CloningBundleFactoryTest { @Rule public ExpectedException thrown = ExpectedException.none(); + @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + private CloningBundleFactory factory = CloningBundleFactory.create(); @Test @@ -76,7 +78,6 @@ public void rootBundleSucceedsIgnoresCoder() { @Test public void bundleWorkingCoderSucceedsClonesOutput() { - TestPipeline p = TestPipeline.create(); PCollection created = p.apply(Create.of(1, 3).withCoder(VarIntCoder.of())); PCollection> kvs = created @@ -101,7 +102,6 @@ public void bundleWorkingCoderSucceedsClonesOutput() { @Test public void keyedBundleWorkingCoderSucceedsClonesOutput() { - TestPipeline p = TestPipeline.create(); PCollection created = p.apply(Create.of(1, 3).withCoder(VarIntCoder.of())); PCollection>> keyed = @@ -130,7 +130,6 @@ public void keyedBundleWorkingCoderSucceedsClonesOutput() { @Test public void bundleEncodeFailsAddFails() { - TestPipeline p = TestPipeline.create(); PCollection pc = p.apply(Create.of().withCoder(new RecordNoEncodeCoder())); UncommittedBundle bundle = factory.createBundle(pc); @@ -142,7 +141,6 @@ public void bundleEncodeFailsAddFails() { @Test public void bundleDecodeFailsAddFails() { - TestPipeline p = TestPipeline.create(); PCollection pc = p.apply(Create.of().withCoder(new RecordNoDecodeCoder())); UncommittedBundle bundle = factory.createBundle(pc); @@ -154,7 +152,6 @@ public void bundleDecodeFailsAddFails() { @Test public void keyedBundleEncodeFailsAddFails() { - TestPipeline p = TestPipeline.create(); PCollection pc = p.apply(Create.of().withCoder(new RecordNoEncodeCoder())); UncommittedBundle bundle = factory.createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), pc); @@ -167,7 +164,6 @@ public void keyedBundleEncodeFailsAddFails() { @Test public void keyedBundleDecodeFailsAddFails() { - TestPipeline p = TestPipeline.create(); PCollection pc = p.apply(Create.of().withCoder(new RecordNoDecodeCoder())); UncommittedBundle bundle = factory.createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), pc); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index c6986c0e6774..736f554210bb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.values.PDone; import org.hamcrest.Matchers; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -47,7 +48,10 @@ */ @RunWith(JUnit4.class) public class CommittedResultTest implements Serializable { - private transient TestPipeline p = TestPipeline.create(); + + @Rule + public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + private transient PCollection created = p.apply(Create.of(1, 2)); private transient AppliedPTransform transform = AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform() { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java index deefc68489dc..35245f4e35da 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java @@ -61,8 +61,15 @@ */ @RunWith(JUnit4.class) public class CopyOnAccessInMemoryStateInternalsTest { + + @Rule public TestPipeline pipeline = TestPipeline.create(); @Rule public ExpectedException thrown = ExpectedException.none(); private String key = "foo"; + + public CopyOnAccessInMemoryStateInternalsTest() { + pipeline = TestPipeline.create(); + } + @Test public void testGetWithEmpty() { CopyOnAccessInMemoryStateInternals internals = @@ -167,7 +174,7 @@ public void testAccumulatorCombiningStateWithUnderlying() throws CannotProvideCo CombineFn sumLongFn = new Sum.SumLongFn(); StateNamespace namespace = new StateNamespaceForTest("foo"); - CoderRegistry reg = TestPipeline.create().getCoderRegistry(); + CoderRegistry reg = pipeline.getCoderRegistry(); StateTag> stateTag = StateTags.combiningValue("summer", sumLongFn.getAccumulatorCoder(reg, reg.getDefaultCoder(Long.class)), sumLongFn); @@ -197,7 +204,7 @@ public void testKeyedAccumulatorCombiningStateWithUnderlying() throws Exception KeyedCombineFn sumLongFn = new Sum.SumLongFn().asKeyedFn(); StateNamespace namespace = new StateNamespaceForTest("foo"); - CoderRegistry reg = TestPipeline.create().getCoderRegistry(); + CoderRegistry reg = pipeline.getCoderRegistry(); StateTag> stateTag = StateTags.keyedCombiningValue( "summer", diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java index b88c9a4775f7..c3bbe2d0222e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java @@ -60,8 +60,9 @@ @RunWith(JUnit4.class) public class DirectGraphVisitorTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Rule public transient TestPipeline p = TestPipeline.create() + .enableAbandonedNodeEnforcement(false); - private transient TestPipeline p = TestPipeline.create(); private transient DirectGraphVisitor visitor = new DirectGraphVisitor(); @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index f11c370ac9d2..bf362048c8d8 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -68,6 +68,7 @@ import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -77,7 +78,6 @@ */ @RunWith(JUnit4.class) public class EvaluationContextTest { - private TestPipeline p; private EvaluationContext context; private PCollection created; @@ -92,13 +92,14 @@ public class EvaluationContextTest { private AppliedPTransform viewProducer; private AppliedPTransform unboundedProducer; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { DirectRunner runner = DirectRunner.fromOptions(PipelineOptionsFactory.create()); - p = TestPipeline.create(); - created = p.apply(Create.of(1, 2, 3)); downstream = created.apply(WithKeys.of("foo")); view = created.apply(View.asIterable()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 9e22c362802b..cda68f0a28c7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -37,6 +37,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.hamcrest.Matchers; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -48,9 +49,11 @@ public class FlattenEvaluatorFactoryTest { private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Test public void testFlattenInMemoryEvaluator() throws Exception { - TestPipeline p = TestPipeline.create(); PCollection left = p.apply("left", Create.of(1, 2, 4)); PCollection right = p.apply("right", Create.of(-1, 2, -4)); PCollectionList list = PCollectionList.of(left).and(right); @@ -118,7 +121,6 @@ public void testFlattenInMemoryEvaluator() throws Exception { @Test public void testFlattenInMemoryEvaluatorWithEmptyPCollectionList() throws Exception { - TestPipeline p = TestPipeline.create(); PCollectionList list = PCollectionList.empty(p); PCollection flattened = list.apply(Flatten.pCollections()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java index f0b29f07ab62..fefafd00dbc7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java @@ -41,6 +41,7 @@ import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -52,9 +53,11 @@ public class GroupByKeyEvaluatorFactoryTest { private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Test public void testInMemoryEvaluator() throws Exception { - TestPipeline p = TestPipeline.create(); KV firstFoo = KV.of("foo", -1); KV secondFoo = KV.of("foo", 1); KV thirdFoo = KV.of("foo", 3); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java index 7efdb3d2b875..94514ad187e0 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java @@ -41,6 +41,7 @@ import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -52,9 +53,11 @@ public class GroupByKeyOnlyEvaluatorFactoryTest { private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Test public void testInMemoryEvaluator() throws Exception { - TestPipeline p = TestPipeline.create(); KV firstFoo = KV.of("foo", -1); KV secondFoo = KV.of("foo", 1); KV thirdFoo = KV.of("foo", 3); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index 6ab8aea2871f..2448078d3cdd 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -46,14 +46,16 @@ */ @RunWith(JUnit4.class) public class ImmutabilityCheckingBundleFactoryTest { + + @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Rule public ExpectedException thrown = ExpectedException.none(); private ImmutabilityCheckingBundleFactory factory; private PCollection created; private PCollection transformed; + @Before public void setup() { - TestPipeline p = TestPipeline.create(); created = p.apply(Create.of().withCoder(ByteArrayCoder.of())); transformed = created.apply(ParDo.of(new IdentityDoFn())); DirectGraphVisitor visitor = new DirectGraphVisitor(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index 1ad6ba6af30c..cd3e9b467a04 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -42,6 +42,8 @@ */ @RunWith(JUnit4.class) public class ImmutabilityEnforcementFactoryTest implements Serializable { + @Rule public transient TestPipeline p = + TestPipeline.create().enableAbandonedNodeEnforcement(false); @Rule public transient ExpectedException thrown = ExpectedException.none(); private transient ImmutabilityEnforcementFactory factory; private transient BundleFactory bundleFactory; @@ -52,7 +54,6 @@ public class ImmutabilityEnforcementFactoryTest implements Serializable { public void setup() { factory = new ImmutabilityEnforcementFactory(); bundleFactory = ImmutableListBundleFactory.create(); - TestPipeline p = TestPipeline.create(); pcollection = p.apply(Create.of("foo".getBytes(), "spamhameggs".getBytes())) .apply( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java index a36c4086eae8..46f02cd58b08 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java @@ -57,6 +57,7 @@ */ @RunWith(JUnit4.class) public class ImmutableListBundleFactoryTest { + @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Rule public ExpectedException thrown = ExpectedException.none(); private ImmutableListBundleFactory bundleFactory = ImmutableListBundleFactory.create(); @@ -66,13 +67,12 @@ public class ImmutableListBundleFactoryTest { @Before public void setup() { - TestPipeline p = TestPipeline.create(); created = p.apply(Create.of(1, 2, 3)); downstream = created.apply(WithKeys.of("foo")); } private void createKeyedBundle(Coder coder, T key) throws Exception { - PCollection pcollection = TestPipeline.create().apply(Create.of(1)); + PCollection pcollection = p.apply("Create", Create.of(1)); StructuralKey skey = StructuralKey.of(key, coder); UncommittedBundle inFlightBundle = bundleFactory.createKeyedBundle(skey, pcollection); @@ -87,9 +87,17 @@ public void keyedWithNullKeyShouldCreateKeyedBundle() throws Exception { } @Test - public void keyedWithKeyShouldCreateKeyedBundle() throws Exception { + public void keyedWithStringKeyShouldCreateKeyedBundle() throws Exception { createKeyedBundle(StringUtf8Coder.of(), "foo"); + } + + @Test + public void keyedWithVarIntKeyShouldCreateKeyedBundle() throws Exception { createKeyedBundle(VarIntCoder.of(), 1234); + } + + @Test + public void keyedWithByteArrayKeyShouldCreateKeyedBundle() throws Exception { createKeyedBundle(ByteArrayCoder.of(), new byte[] {0, 2, 4, 99}); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index 0852cd3f544d..eef3375b445e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableSet; import java.util.Collections; import java.util.Set; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -53,11 +52,12 @@ public class KeyedPValueTrackingVisitorTest { @Rule public ExpectedException thrown = ExpectedException.none(); private KeyedPValueTrackingVisitor visitor; - private Pipeline p; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Before public void setup() { - p = TestPipeline.create(); + @SuppressWarnings("rawtypes") Set> producesKeyed = ImmutableSet.>of(PrimitiveKeyer.class, CompositeKeyer.class); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index d48ac142ad6d..1a3207b5b9f7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -54,6 +54,7 @@ import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -72,10 +73,12 @@ public class ParDoEvaluatorTest { private List> sideOutputTags; private BundleFactory bundleFactory; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { MockitoAnnotations.initMocks(this); - TestPipeline p = TestPipeline.create(); inputPc = p.apply(Create.of(1, 2, 3)); mainOutputTag = new TupleTag() {}; sideOutputTags = TupleTagList.empty().getAll(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index cc7d88a39e7e..183decde6dbb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -95,13 +95,15 @@ public String toString() { } }; + @Rule + public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public ExpectedException thrown = ExpectedException.none(); @Mock private EvaluationContext context; - private TestPipeline pipeline; private SideInputContainer container; @@ -114,7 +116,6 @@ public String toString() { @Before public void setup() { MockitoAnnotations.initMocks(this); - pipeline = TestPipeline.create(); PCollection create = pipeline.apply("forBaseCollection", Create.of(1, 2, 3, 4)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 326310bf6db9..d312aa36f770 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -67,6 +67,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -91,6 +92,10 @@ public class StatefulParDoEvaluatorFactoryTest implements Serializable { private static final BundleFactory BUNDLE_FACTORY = ImmutableListBundleFactory.create(); + @Rule + public transient TestPipeline pipeline = + TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { MockitoAnnotations.initMocks(this); @@ -106,7 +111,6 @@ public void setup() { public void windowCleanupScheduled() throws Exception { // To test the factory, first we set up a pipeline and then we use the constructed // pipeline to create the right parameters to pass to the factory - TestPipeline pipeline = TestPipeline.create(); final String stateId = "my-state-id"; @@ -208,7 +212,6 @@ public void process(ProcessContext c) {} public void testUnprocessedElements() throws Exception { // To test the factory, first we set up a pipeline and then we use the constructed // pipeline to create the right parameters to pass to the factory - TestPipeline pipeline = TestPipeline.create(); final String stateId = "my-state-id"; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java index d3a2cca84444..0d94b7ae397e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.values.PCollection; import org.hamcrest.Matchers; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -44,9 +45,11 @@ public class StepTransformResultTest { private BundleFactory bundleFactory; private PCollection pc; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { - TestPipeline p = TestPipeline.create(); pc = p.apply(Create.of(1, 2, 3)); transform = DirectGraphs.getGraph(p).getProducer(pc); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index 6bb86233a0dc..c5b3b3d14a17 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -41,6 +41,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -52,6 +53,9 @@ public class TestStreamEvaluatorFactoryTest { private BundleFactory bundleFactory; private EvaluationContext context; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { context = mock(EvaluationContext.class); @@ -62,7 +66,6 @@ public void setup() { /** Demonstrates that returned evaluators produce elements in sequence. */ @Test public void producesElementsInSequence() throws Exception { - TestPipeline p = TestPipeline.create(); PCollection streamVals = p.apply( TestStream.create(VarIntCoder.of()) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java index 4ad22bc3af49..e66ffcf81af9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java @@ -73,6 +73,9 @@ public class TransformExecutorTest { @Mock private EvaluationContext evaluationContext; @Mock private TransformEvaluatorRegistry registry; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { MockitoAnnotations.initMocks(this); @@ -85,7 +88,6 @@ public void setup() { evaluatorCompleted = new CountDownLatch(1); completionCallback = new RegisteringCompletionCallback(evaluatorCompleted); - TestPipeline p = TestPipeline.create(); created = p.apply(Create.of("foo", "spam", "third")); PCollection> downstream = created.apply(WithKeys.of(3)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index dd36a2f8c1f7..92d668e4f7e1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -72,6 +72,7 @@ import org.joda.time.Instant; import org.joda.time.ReadableInstant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -93,10 +94,12 @@ public class UnboundedReadEvaluatorFactoryTest { private UnboundedSource source; private DirectGraph graph; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { source = CountingSource.unboundedWithTimestampFn(new LongToInstantFn()); - TestPipeline p = TestPipeline.create(); longs = p.apply(Read.from(source)); context = mock(EvaluationContext.class); @@ -190,7 +193,6 @@ public void unboundedSourceWithDuplicatesMultipleCalls() throws Exception { new TestUnboundedSource<>(BigEndianLongCoder.of(), outputs); source.dedupes = true; - TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); AppliedPTransform sourceTransform = getProducer(pcollection); @@ -231,7 +233,6 @@ public void unboundedSourceWithDuplicatesMultipleCalls() throws Exception { @Test public void noElementsAvailableReaderIncludedInResidual() throws Exception { - TestPipeline p = TestPipeline.create(); // Read with a very slow rate so by the second read there are no more elements PCollection pcollection = p.apply(Read.from(new TestUnboundedSource<>(VarLongCoder.of(), 1L))); @@ -291,7 +292,6 @@ public void evaluatorReusesReader() throws Exception { TestUnboundedSource source = new TestUnboundedSource<>(BigEndianLongCoder.of(), elems.toArray(new Long[0])); - TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); DirectGraph graph = DirectGraphs.getGraph(p); AppliedPTransform sourceTransform = @@ -337,7 +337,6 @@ public void evaluatorClosesReaderAndResumesFromCheckpoint() throws Exception { TestUnboundedSource source = new TestUnboundedSource<>(BigEndianLongCoder.of(), elems.toArray(new Long[0])); - TestPipeline p = TestPipeline.create(); PCollection pcollection = p.apply(Read.from(source)); AppliedPTransform sourceTransform = DirectGraphs.getGraph(p).getProducer(pcollection); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index 7c080099675a..6baf55a4ece2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -52,9 +53,11 @@ public class ViewEvaluatorFactoryTest { private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Test public void testInMemoryEvaluator() throws Exception { - TestPipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of("foo", "bar")); CreatePCollectionView> createView = diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java index acdabb6b26bd..8d6e73f54f41 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java @@ -37,6 +37,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -51,9 +52,11 @@ public class WatermarkCallbackExecutorTest { private AppliedPTransform create; private AppliedPTransform sum; + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { - TestPipeline p = TestPipeline.create(); PCollection created = p.apply(Create.of(1, 2, 3)); PCollection summed = created.apply(Sum.integersGlobally()); DirectGraph graph = DirectGraphs.getGraph(p); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index eb4d0cdc9966..abc8a28722d5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -70,6 +70,7 @@ import org.joda.time.Instant; import org.joda.time.ReadableInstant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -94,9 +95,11 @@ public class WatermarkManagerTest implements Serializable { private transient BundleFactory bundleFactory; private DirectGraph graph; + @Rule + public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { - TestPipeline p = TestPipeline.create(); createdInts = p.apply("createdInts", Create.of(1, 2, 3)); @@ -278,7 +281,6 @@ public void getWatermarkForMultiInputTransform() { */ @Test public void getWatermarkMultiIdenticalInput() { - TestPipeline p = TestPipeline.create(); PCollection created = p.apply(Create.of(1, 2, 3)); PCollection multiConsumer = PCollectionList.of(created).and(created).apply(Flatten.pCollections()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index 66c28ce612a2..9d0c68d46ba5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -52,6 +52,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -96,10 +97,12 @@ public class WindowEvaluatorFactoryTest { ImmutableList.of(GlobalWindow.INSTANCE, intervalWindow1, intervalWindow2), PaneInfo.NO_FIRING); + @Rule + public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Before public void setup() { MockitoAnnotations.initMocks(this); - TestPipeline p = TestPipeline.create(); input = p.apply(Create.of(1L, 2L, 3L)); bundleFactory = ImmutableListBundleFactory.create(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index aeb75ed88968..a8c4c02444e1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -69,6 +69,7 @@ public class WriteWithShardingFactoryTest { public static final int INPUT_SIZE = 10000; @Rule public TemporaryFolder tmp = new TemporaryFolder(); private WriteWithShardingFactory factory = new WriteWithShardingFactory<>(); + @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Test public void dynamicallyReshardedWrite() throws Exception { @@ -81,7 +82,6 @@ public void dynamicallyReshardedWrite() throws Exception { String fileName = "resharded_write"; String outputPath = tmp.getRoot().getAbsolutePath(); String targetLocation = IOChannelUtils.resolve(outputPath, fileName); - TestPipeline p = TestPipeline.create(); // TextIO is implemented in terms of the Write PTransform. When sharding is not specified, // resharding should be automatically applied p.apply(Create.of(strs)).apply(TextIO.Write.to(targetLocation)); @@ -134,7 +134,7 @@ public void withNoShardingSpecifiedReturnsNewTransform() { public void keyBasedOnCountFnWithOneElement() throws Exception { PCollectionView elementCountView = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); KeyBasedOnCountFn fn = new KeyBasedOnCountFn<>(elementCountView, 0); DoFnTester> fnTester = DoFnTester.of(fn); @@ -149,7 +149,7 @@ public void keyBasedOnCountFnWithOneElement() throws Exception { public void keyBasedOnCountFnWithTwoElements() throws Exception { PCollectionView elementCountView = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); KeyBasedOnCountFn fn = new KeyBasedOnCountFn<>(elementCountView, 0); DoFnTester> fnTester = DoFnTester.of(fn); @@ -167,7 +167,7 @@ public void keyBasedOnCountFnWithTwoElements() throws Exception { public void keyBasedOnCountFnFewElementsThreeShards() throws Exception { PCollectionView elementCountView = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); KeyBasedOnCountFn fn = new KeyBasedOnCountFn<>(elementCountView, 0); DoFnTester> fnTester = DoFnTester.of(fn); @@ -191,7 +191,7 @@ public Integer apply(KV input) { public void keyBasedOnCountFnManyElements() throws Exception { PCollectionView elementCountView = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); KeyBasedOnCountFn fn = new KeyBasedOnCountFn<>(elementCountView, 0); DoFnTester> fnTester = DoFnTester.of(fn); @@ -214,7 +214,7 @@ public void keyBasedOnCountFnManyElements() throws Exception { public void keyBasedOnCountFnFewElementsExtraShards() throws Exception { PCollectionView elementCountView = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); KeyBasedOnCountFn fn = new KeyBasedOnCountFn<>(elementCountView, 10); DoFnTester> fnTester = DoFnTester.of(fn); @@ -238,7 +238,7 @@ public void keyBasedOnCountFnFewElementsExtraShards() throws Exception { public void keyBasedOnCountFnManyElementsExtraShards() throws Exception { PCollectionView elementCountView = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0L, VarLongCoder.of()); KeyBasedOnCountFn fn = new KeyBasedOnCountFn<>(elementCountView, 3); DoFnTester> fnTester = DoFnTester.of(fn); From 75a4c918346b5a04213a54bf7d1bf6507655342a Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Mon, 19 Dec 2016 23:54:47 +0200 Subject: [PATCH 246/279] Migrated the beam-sdks-java-core module to TestPipeline as a JUnit rule. Plus, fixed some checkstyle errors from previous modules' migration. --- .../UnboundedReadFromBoundedSourceTest.java | 1 - .../direct/CloningBundleFactoryTest.java | 2 +- ...opyOnAccessInMemoryStateInternalsTest.java | 6 +- ...ImmutabilityCheckingBundleFactoryTest.java | 2 +- .../ImmutableListBundleFactoryTest.java | 2 +- .../direct/WriteWithShardingFactoryTest.java | 2 +- .../org/apache/beam/sdk/PipelineTest.java | 37 +++--- .../apache/beam/sdk/coders/AvroCoderTest.java | 11 +- .../beam/sdk/coders/CoderRegistryTest.java | 6 +- .../sdk/coders/SerializableCoderTest.java | 7 +- .../org/apache/beam/sdk/io/AvroIOTest.java | 16 +-- .../BoundedReadFromUnboundedSourceTest.java | 6 +- .../beam/sdk/io/CompressedSourceTest.java | 12 +- .../apache/beam/sdk/io/CountingInputTest.java | 12 +- .../beam/sdk/io/CountingSourceTest.java | 13 +- .../beam/sdk/io/FileBasedSourceTest.java | 4 +- .../beam/sdk/io/PubsubUnboundedSinkTest.java | 10 +- .../sdk/io/PubsubUnboundedSourceTest.java | 12 +- .../org/apache/beam/sdk/io/TextIOTest.java | 29 +++-- .../org/apache/beam/sdk/io/WriteTest.java | 2 +- .../org/apache/beam/sdk/io/XmlSourceTest.java | 10 +- .../options/ProxyInvocationHandlerTest.java | 5 +- .../sdk/runners/TransformHierarchyTest.java | 6 +- .../beam/sdk/runners/TransformTreeTest.java | 11 +- .../beam/sdk/testing/GatherAllPanesTest.java | 7 +- .../apache/beam/sdk/testing/PAssertTest.java | 32 ++--- .../beam/sdk/testing/TestStreamTest.java | 7 +- .../transforms/ApproximateQuantilesTest.java | 12 +- .../sdk/transforms/ApproximateUniqueTest.java | 6 +- .../beam/sdk/transforms/CombineFnsTest.java | 5 +- .../beam/sdk/transforms/CombineTest.java | 25 +--- .../apache/beam/sdk/transforms/CountTest.java | 13 +- .../beam/sdk/transforms/CreateTest.java | 27 +--- .../beam/sdk/transforms/DistinctTest.java | 12 +- .../apache/beam/sdk/transforms/DoFnTest.java | 4 +- .../beam/sdk/transforms/DoFnTesterTest.java | 6 +- .../beam/sdk/transforms/FilterTest.java | 18 +-- .../sdk/transforms/FlatMapElementsTest.java | 10 +- .../beam/sdk/transforms/FlattenTest.java | 35 +----- .../beam/sdk/transforms/GroupByKeyTest.java | 30 ++--- .../apache/beam/sdk/transforms/KeysTest.java | 9 +- .../beam/sdk/transforms/KvSwapTest.java | 9 +- .../beam/sdk/transforms/LatestTest.java | 12 +- .../beam/sdk/transforms/MapElementsTest.java | 14 +-- .../sdk/transforms/ParDoLifecycleTest.java | 17 +-- .../apache/beam/sdk/transforms/ParDoTest.java | 118 +++++++----------- .../beam/sdk/transforms/PartitionTest.java | 8 +- .../apache/beam/sdk/transforms/RegexTest.java | 25 +--- .../beam/sdk/transforms/SampleTest.java | 34 ++--- .../sdk/transforms/SplittableDoFnTest.java | 12 +- .../apache/beam/sdk/transforms/TopTest.java | 15 ++- .../beam/sdk/transforms/ValuesTest.java | 7 +- .../apache/beam/sdk/transforms/ViewTest.java | 84 ++++--------- .../beam/sdk/transforms/WithKeysTest.java | 8 +- .../sdk/transforms/WithTimestampsTest.java | 9 +- .../sdk/transforms/join/CoGroupByKeyTest.java | 11 +- .../sdk/transforms/windowing/WindowTest.java | 22 ++-- .../transforms/windowing/WindowingTest.java | 11 +- .../apache/beam/sdk/util/ReshuffleTest.java | 11 +- .../beam/sdk/values/PCollectionTupleTest.java | 12 +- .../org/apache/beam/sdk/values/PDoneTest.java | 9 +- .../beam/sdk/values/TypedPValueTest.java | 10 +- 62 files changed, 353 insertions(+), 587 deletions(-) diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java index 86450f275e9c..0f09cd1439ed 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java @@ -36,7 +36,6 @@ import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.CheckpointCoder; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java index e5299a2b3b85..505d3a2410e5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java @@ -62,7 +62,7 @@ @RunWith(JUnit4.class) public class CloningBundleFactoryTest { @Rule public ExpectedException thrown = ExpectedException.none(); - @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); private CloningBundleFactory factory = CloningBundleFactory.create(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java index 35245f4e35da..12ef66c692dd 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java @@ -62,14 +62,10 @@ @RunWith(JUnit4.class) public class CopyOnAccessInMemoryStateInternalsTest { - @Rule public TestPipeline pipeline = TestPipeline.create(); + @Rule public final TestPipeline pipeline = TestPipeline.create(); @Rule public ExpectedException thrown = ExpectedException.none(); private String key = "foo"; - public CopyOnAccessInMemoryStateInternalsTest() { - pipeline = TestPipeline.create(); - } - @Test public void testGetWithEmpty() { CopyOnAccessInMemoryStateInternals internals = diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index 2448078d3cdd..eccb3a66de48 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -47,7 +47,7 @@ @RunWith(JUnit4.class) public class ImmutabilityCheckingBundleFactoryTest { - @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Rule public ExpectedException thrown = ExpectedException.none(); private ImmutabilityCheckingBundleFactory factory; private PCollection created; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java index 46f02cd58b08..3327ccd8ef09 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java @@ -57,7 +57,7 @@ */ @RunWith(JUnit4.class) public class ImmutableListBundleFactoryTest { - @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Rule public ExpectedException thrown = ExpectedException.none(); private ImmutableListBundleFactory bundleFactory = ImmutableListBundleFactory.create(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index a8c4c02444e1..7432e61ae196 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -69,7 +69,7 @@ public class WriteWithShardingFactoryTest { public static final int INPUT_SIZE = 10000; @Rule public TemporaryFolder tmp = new TemporaryFolder(); private WriteWithShardingFactory factory = new WriteWithShardingFactory<>(); - @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @Test public void dynamicallyReshardedWrite() throws Exception { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index fea1554376b0..d8e4ef49827f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -62,6 +62,7 @@ @RunWith(JUnit4.class) public class PipelineTest { + @Rule public final TestPipeline pipeline = TestPipeline.create(); @Rule public ExpectedLogs logged = ExpectedLogs.none(Pipeline.class); @Rule public ExpectedException thrown = ExpectedException.none(); @@ -128,8 +129,7 @@ public void testMultipleApply() { PTransform, PCollection> myTransform = addSuffix("+"); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(ImmutableList.of("a", "b"))); + PCollection input = pipeline.apply(Create.of(ImmutableList.of("a", "b"))); PCollection left = input.apply("Left1", myTransform).apply("Left2", myTransform); PCollection right = input.apply("Right", myTransform); @@ -139,7 +139,7 @@ public void testMultipleApply() { PAssert.that(both).containsInAnyOrder("a++", "b++", "a+", "b+"); - p.run(); + pipeline.run(); } private static PTransform, PCollection> addSuffix( @@ -162,35 +162,36 @@ public void testToString() { @Test public void testStableUniqueNameOff() { - Pipeline p = TestPipeline.create(); - p.getOptions().setStableUniqueNames(CheckEnabled.OFF); + pipeline.enableAbandonedNodeEnforcement(false); + + pipeline.getOptions().setStableUniqueNames(CheckEnabled.OFF); - p.apply(Create.of(5, 6, 7)); - p.apply(Create.of(5, 6, 7)); + pipeline.apply(Create.of(5, 6, 7)); + pipeline.apply(Create.of(5, 6, 7)); logged.verifyNotLogged("does not have a stable unique name."); } @Test public void testStableUniqueNameWarning() { - Pipeline p = TestPipeline.create(); - p.getOptions().setStableUniqueNames(CheckEnabled.WARNING); + pipeline.enableAbandonedNodeEnforcement(false); - p.apply(Create.of(5, 6, 7)); - p.apply(Create.of(5, 6, 7)); + pipeline.getOptions().setStableUniqueNames(CheckEnabled.WARNING); + + pipeline.apply(Create.of(5, 6, 7)); + pipeline.apply(Create.of(5, 6, 7)); logged.verifyWarn("does not have a stable unique name."); } @Test public void testStableUniqueNameError() { - Pipeline p = TestPipeline.create(); - p.getOptions().setStableUniqueNames(CheckEnabled.ERROR); + pipeline.getOptions().setStableUniqueNames(CheckEnabled.ERROR); - p.apply(Create.of(5, 6, 7)); + pipeline.apply(Create.of(5, 6, 7)); thrown.expectMessage("does not have a stable unique name."); - p.apply(Create.of(5, 6, 7)); + pipeline.apply(Create.of(5, 6, 7)); } /** @@ -199,7 +200,6 @@ public void testStableUniqueNameError() { @Test @Category(RunnableOnService.class) public void testIdentityTransform() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3, 4)) @@ -223,8 +223,6 @@ public T expand(T input) { @Test @Category(RunnableOnService.class) public void testTupleProjectionTransform() throws Exception { - Pipeline pipeline = TestPipeline.create(); - PCollection input = pipeline .apply(Create.of(1, 2, 3, 4)); @@ -258,8 +256,6 @@ public PCollection expand(PCollectionTuple input) { @Test @Category(RunnableOnService.class) public void testTupleInjectionTransform() throws Exception { - Pipeline pipeline = TestPipeline.create(); - PCollection input = pipeline .apply(Create.of(1, 2, 3, 4)); @@ -292,7 +288,6 @@ public PCollectionTuple expand(PCollection input) { @Test @Category(NeedsRunner.class) public void testEmptyPipeline() throws Exception { - Pipeline pipeline = TestPipeline.create(); pipeline.run(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java index adfa0d293677..60dc07a13af8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java @@ -55,7 +55,6 @@ import org.apache.avro.reflect.Union; import org.apache.avro.specific.SpecificData; import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.testing.CoderProperties; @@ -73,6 +72,7 @@ import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -143,6 +143,9 @@ public void processElement(ProcessContext c) { } } + @Rule + public TestPipeline pipeline = TestPipeline.create(); + @Test public void testAvroCoderEncoding() throws Exception { AvroCoder coder = AvroCoder.of(Pojo.class); @@ -287,17 +290,15 @@ public void testEncodingNotBuffered() throws Exception { @Test @Category(NeedsRunner.class) public void testDefaultCoder() throws Exception { - Pipeline p = TestPipeline.create(); - // Use MyRecord as input and output types without explicitly specifying // a coder (this uses the default coders, which may not be AvroCoder). PCollection output = - p.apply(Create.of(new Pojo("hello", 1), new Pojo("world", 2))) + pipeline.apply(Create.of(new Pojo("hello", 1), new Pojo("world", 2))) .apply(ParDo.of(new GetTextFn())); PAssert.that(output) .containsInAnyOrder("hello", "world"); - p.run(); + pipeline.run(); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java index d7badab10d49..8c0e584f6ac1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderRegistry.IncompatibleCoderException; import org.apache.beam.sdk.coders.protobuf.ProtoCoder; import org.apache.beam.sdk.testing.NeedsRunner; @@ -62,6 +61,9 @@ @RunWith(JUnit4.class) public class CoderRegistryTest { + @Rule + public TestPipeline pipeline = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); @@ -414,7 +416,6 @@ private static class TestSerializableGenericClass coder = IterableCoder @@ -136,7 +139,7 @@ public void testNullEquals() { @Test @Category(NeedsRunner.class) public void testDefaultCoder() throws Exception { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(true); // Use MyRecord as input and output types without explicitly specifying // a coder (this uses the default coders, which may not be diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java index 41a630fd29fc..b669968f7f86 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java @@ -74,6 +74,10 @@ */ @RunWith(JUnit4.class) public class AvroIOTest { + + @Rule + public TestPipeline p = TestPipeline.create(); + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -135,7 +139,6 @@ public boolean equals(Object other) { @Test @Category(NeedsRunner.class) public void testAvroIOWriteAndReadASingleFile() throws Throwable { - TestPipeline p = TestPipeline.create(); List values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); @@ -146,7 +149,6 @@ public void testAvroIOWriteAndReadASingleFile() throws Throwable { .withSchema(GenericClass.class)); p.run(); - p = TestPipeline.create(); PCollection input = p .apply(AvroIO.Read.from(outputFile.getAbsolutePath()).withSchema(GenericClass.class)); @@ -158,7 +160,6 @@ public void testAvroIOWriteAndReadASingleFile() throws Throwable { @SuppressWarnings("unchecked") @Category(NeedsRunner.class) public void testAvroIOCompressedWriteAndReadASingleFile() throws Throwable { - TestPipeline p = TestPipeline.create(); List values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); @@ -170,7 +171,6 @@ public void testAvroIOCompressedWriteAndReadASingleFile() throws Throwable { .withSchema(GenericClass.class)); p.run(); - p = TestPipeline.create(); PCollection input = p .apply(AvroIO.Read .from(outputFile.getAbsolutePath()) @@ -187,7 +187,6 @@ public void testAvroIOCompressedWriteAndReadASingleFile() throws Throwable { @SuppressWarnings("unchecked") @Category(NeedsRunner.class) public void testAvroIONullCodecWriteAndReadASingleFile() throws Throwable { - TestPipeline p = TestPipeline.create(); List values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); @@ -199,7 +198,6 @@ public void testAvroIONullCodecWriteAndReadASingleFile() throws Throwable { .withCodec(CodecFactory.nullCodec())); p.run(); - p = TestPipeline.create(); PCollection input = p .apply(AvroIO.Read .from(outputFile.getAbsolutePath()) @@ -257,7 +255,6 @@ public boolean equals(Object other) { @Test @Category(NeedsRunner.class) public void testAvroIOWriteAndReadSchemaUpgrade() throws Throwable { - TestPipeline p = TestPipeline.create(); List values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); @@ -270,7 +267,7 @@ public void testAvroIOWriteAndReadSchemaUpgrade() throws Throwable { List expected = ImmutableList.of(new GenericClassV2(3, "hi", null), new GenericClassV2(5, "bar", null)); - p = TestPipeline.create(); + PCollection input = p .apply(AvroIO.Read.from(outputFile.getAbsolutePath()).withSchema(GenericClassV2.class)); @@ -321,7 +318,6 @@ public void testWriteWithSerDeCustomXZCodec() throws Exception { @SuppressWarnings("unchecked") @Category(NeedsRunner.class) public void testMetdata() throws Exception { - TestPipeline p = TestPipeline.create(); List values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); @@ -348,7 +344,7 @@ public void testMetdata() throws Exception { private void runTestWrite(String[] expectedElements, int numShards) throws IOException { File baseOutputFile = new File(tmpFolder.getRoot(), "prefix"); String outputFilePrefix = baseOutputFile.getAbsolutePath(); - TestPipeline p = TestPipeline.create(); + Bound write = AvroIO.Write.to(outputFilePrefix).withSchema(String.class); if (numShards > 1) { write = write.withNumShards(numShards); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java index 4d7814cf8518..d49873e7ec98 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.dataflow.TestCountingSource; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -46,6 +46,9 @@ public class BoundedReadFromUnboundedSourceTest implements Serializable{ private static final int NUM_RECORDS = 100; + @Rule + public transient TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testNoDedup() throws Exception { @@ -112,7 +115,6 @@ public Void apply(Iterable> input) { } private void test(boolean dedup, boolean timeBound) throws Exception { - Pipeline p = TestPipeline.create(); TestCountingSource source = new TestCountingSource(Integer.MAX_VALUE).withoutSplitting(); if (dedup) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java index f8769eab411d..3871159f68dc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java @@ -47,7 +47,6 @@ import java.util.Random; import java.util.zip.GZIPOutputStream; import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; @@ -80,6 +79,10 @@ */ @RunWith(JUnit4.class) public class CompressedSourceTest { + + @Rule + public TestPipeline p = TestPipeline.create(); + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -199,8 +202,6 @@ public void testReadConcatenatedGzip() throws IOException { os.write(totalGz); } - Pipeline p = TestPipeline.create(); - CompressedSource source = CompressedSource.from(new ByteSource(tmpFile.getAbsolutePath(), 1)) .withDecompression(CompressionMode.GZIP); @@ -274,8 +275,6 @@ public void testHeterogeneousCompression() throws Exception { String filePattern = new File(tmpFolder.getRoot().toString(), baseName + ".*").toString(); - Pipeline p = TestPipeline.create(); - CompressedSource source = CompressedSource.from(new ByteSource(filePattern, 1)); PCollection output = p.apply(Read.from(source)); @@ -395,8 +394,6 @@ public void testCompressedReadMultipleFiles() throws Exception { expected.addAll(Bytes.asList(generated)); } - Pipeline p = TestPipeline.create(); - CompressedSource source = CompressedSource.from(new ByteSource(filePattern, 1)) .withDecompression(CompressionMode.GZIP); @@ -476,7 +473,6 @@ private void runReadTest(byte[] input, private void verifyReadContents(byte[] expected, File inputFile, @Nullable DecompressingChannelFactory decompressionFactory) { - Pipeline p = TestPipeline.create(); CompressedSource source = CompressedSource.from(new ByteSource(inputFile.toPath().toString(), 1)); if (decompressionFactory != null) { 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 dfc49193fa9b..f23ee766253e 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 @@ -21,7 +21,6 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput.UnboundedCountingInput; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -39,6 +38,7 @@ import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -66,10 +66,12 @@ public static void addCountingAsserts(PCollection input, long start, long .isEqualTo(end - 1); } + @Rule + public TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testBoundedInput() { - Pipeline p = TestPipeline.create(); long numElements = 1000; PCollection input = p.apply(CountingInput.upTo(numElements)); @@ -80,7 +82,6 @@ public void testBoundedInput() { @Test @Category(RunnableOnService.class) public void testEmptyBoundedInput() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(CountingInput.upTo(0)); PAssert.that(input).empty(); @@ -90,7 +91,6 @@ public void testEmptyBoundedInput() { @Test @Category(RunnableOnService.class) public void testEmptyBoundedInputSubrange() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(CountingInput.forSubrange(42, 42)); PAssert.that(input).empty(); @@ -101,7 +101,6 @@ public void testEmptyBoundedInputSubrange() { @Test @Category(RunnableOnService.class) public void testBoundedInputSubrange() { - Pipeline p = TestPipeline.create(); long start = 10; long end = 1000; PCollection input = p.apply(CountingInput.forSubrange(start, end)); @@ -128,7 +127,6 @@ public void testBoundedDisplayDataSubrange() { @Test @Category(RunnableOnService.class) public void testUnboundedInput() { - Pipeline p = TestPipeline.create(); long numElements = 1000; PCollection input = p.apply(CountingInput.unbounded().withMaxNumRecords(numElements)); @@ -140,7 +138,6 @@ public void testUnboundedInput() { @Test @Category(NeedsRunner.class) public void testUnboundedInputRate() { - Pipeline p = TestPipeline.create(); long numElements = 5000; long elemsPerPeriod = 10L; @@ -169,7 +166,6 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(RunnableOnService.class) public void testUnboundedInputTimestamps() { - Pipeline p = TestPipeline.create(); long numElements = 1000; PCollection input = 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 5eccde6f2282..dfd0949046a2 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 @@ -25,7 +25,6 @@ import java.io.IOException; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.CountingSource.CounterMark; import org.apache.beam.sdk.io.CountingSource.UnboundedCountingSource; @@ -48,6 +47,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -79,10 +79,12 @@ public static void addCountingAsserts(PCollection input, long numElements) .isEqualTo(numElements - 1); } + @Rule + public TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testBoundedSource() { - Pipeline p = TestPipeline.create(); long numElements = 1000; PCollection input = p.apply(Read.from(CountingSource.upTo(numElements))); @@ -93,7 +95,6 @@ public void testBoundedSource() { @Test @Category(RunnableOnService.class) public void testEmptyBoundedSource() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Read.from(CountingSource.upTo(0))); PAssert.that(input).empty(); @@ -103,7 +104,6 @@ public void testEmptyBoundedSource() { @Test @Category(RunnableOnService.class) public void testBoundedSourceSplits() throws Exception { - Pipeline p = TestPipeline.create(); long numElements = 1000; long numSplits = 10; long splitSizeBytes = numElements * 8 / numSplits; // 8 bytes per long element. @@ -157,7 +157,6 @@ public void testProgress() throws IOException { @Test @Category(RunnableOnService.class) public void testUnboundedSource() { - Pipeline p = TestPipeline.create(); long numElements = 1000; PCollection input = p @@ -177,7 +176,6 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(RunnableOnService.class) public void testUnboundedSourceTimestamps() { - Pipeline p = TestPipeline.create(); long numElements = 1000; PCollection input = p.apply( @@ -197,7 +195,6 @@ public void testUnboundedSourceTimestamps() { @Test @Category(NeedsRunner.class) public void testUnboundedSourceWithRate() { - Pipeline p = TestPipeline.create(); Duration period = Duration.millis(5); long numElements = 1000L; @@ -232,7 +229,6 @@ public void testUnboundedSourceWithRate() { @Test @Category(RunnableOnService.class) public void testUnboundedSourceSplits() throws Exception { - Pipeline p = TestPipeline.create(); long numElements = 1000; int numSplits = 10; @@ -257,7 +253,6 @@ public void testUnboundedSourceSplits() throws Exception { @Test @Category(NeedsRunner.class) public void testUnboundedSourceRateSplits() throws Exception { - Pipeline p = TestPipeline.create(); int elementsPerPeriod = 10; Duration period = Duration.millis(5); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java index f4b857473239..f709e22f90bf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java @@ -42,7 +42,6 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Random; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileBasedSource.FileBasedReader; @@ -73,6 +72,7 @@ public class FileBasedSourceTest { Random random = new Random(0L); + @Rule public final TestPipeline p = TestPipeline.create(); @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @Rule public ExpectedException thrown = ExpectedException.none(); @@ -719,7 +719,6 @@ public void testReadAllSplitsOfSingleFile() throws Exception { @Test @Category(NeedsRunner.class) public void testDataflowFile() throws IOException { - Pipeline p = TestPipeline.create(); List data = createStringDataset(3, 50); String fileName = "file"; @@ -735,7 +734,6 @@ public void testDataflowFile() throws IOException { @Test @Category(NeedsRunner.class) public void testDataflowFilePattern() throws IOException { - Pipeline p = TestPipeline.create(); List data1 = createStringDataset(3, 50); File file1 = createFileWithData("file1", data1); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java index 518136fa1d7b..5bc166494ee3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSinkTest.java @@ -39,6 +39,7 @@ import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -67,6 +68,9 @@ private String getRecordId(String data) { return Hashing.murmur3_128().hashBytes(data.getBytes()).toString(); } + @Rule + public TestPipeline p = TestPipeline.create(); + @Test public void saneCoder() throws Exception { OutgoingMessage message = new OutgoingMessage(DATA.getBytes(), TIMESTAMP, getRecordId(DATA)); @@ -88,7 +92,7 @@ public void sendOneMessage() throws IOException { new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); - TestPipeline p = TestPipeline.create(); + p.apply(Create.of(ImmutableList.of(DATA))) .apply(ParDo.of(new Stamp())) .apply(sink); @@ -117,7 +121,7 @@ public void sendMoreThanOneBatchByNumMessages() throws IOException { new PubsubUnboundedSink<>(factory, StaticValueProvider.of(TOPIC), StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); - TestPipeline p = TestPipeline.create(); + p.apply(Create.of(data)) .apply(ParDo.of(new Stamp())) .apply(sink); @@ -153,7 +157,7 @@ public void sendMoreThanOneBatchByByteSize() throws IOException { StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL, NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); - TestPipeline p = TestPipeline.create(); + p.apply(Create.of(data)) .apply(ParDo.of(new Stamp())) .apply(sink); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java index f6165c507526..601e2c819b2e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubUnboundedSourceTest.java @@ -57,6 +57,7 @@ import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; import org.joda.time.Instant; import org.junit.After; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -82,6 +83,9 @@ public class PubsubUnboundedSourceTest { private PubsubTestClientFactory factory; private PubsubSource primSource; + @Rule + public TestPipeline p = TestPipeline.create(); + private void setupOneMessage(Iterable incoming) { now = new AtomicLong(REQ_TIME); clock = new Clock() { @@ -124,7 +128,6 @@ public void checkpointCoderIsSane() throws Exception { @Test public void readOneMessage() throws IOException { setupOneMessage(); - TestPipeline p = TestPipeline.create(); PubsubReader reader = primSource.createReader(p.getOptions(), null); // Read one message. assertTrue(reader.start()); @@ -139,7 +142,6 @@ public void readOneMessage() throws IOException { @Test public void timeoutAckAndRereadOneMessage() throws IOException { setupOneMessage(); - TestPipeline p = TestPipeline.create(); PubsubReader reader = primSource.createReader(p.getOptions(), null); PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient(); assertTrue(reader.start()); @@ -160,7 +162,6 @@ public void timeoutAckAndRereadOneMessage() throws IOException { @Test public void extendAck() throws IOException { setupOneMessage(); - TestPipeline p = TestPipeline.create(); PubsubReader reader = primSource.createReader(p.getOptions(), null); PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient(); // Pull the first message but don't take a checkpoint for it. @@ -183,7 +184,6 @@ public void extendAck() throws IOException { @Test public void timeoutAckExtensions() throws IOException { setupOneMessage(); - TestPipeline p = TestPipeline.create(); PubsubReader reader = primSource.createReader(p.getOptions(), null); PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient(); // Pull the first message but don't take a checkpoint for it. @@ -220,7 +220,6 @@ public void multipleReaders() throws IOException { incoming.add(new IncomingMessage(data.getBytes(), TIMESTAMP, 0, ackid, RECORD_ID)); } setupOneMessage(incoming); - TestPipeline p = TestPipeline.create(); PubsubReader reader = primSource.createReader(p.getOptions(), null); // Consume two messages, only read one. assertTrue(reader.start()); @@ -281,7 +280,6 @@ public void readManyMessages() throws IOException { } setupOneMessage(incoming); - TestPipeline p = TestPipeline.create(); PubsubReader reader = primSource.createReader(p.getOptions(), null); PubsubTestClient pubsubClient = (PubsubTestClient) reader.getPubsubClient(); @@ -342,7 +340,6 @@ public void noSubscriptionSplitIntoBundlesGeneratesSubscription() throws Excepti null); assertThat(source.getSubscription(), nullValue()); - TestPipeline.create().apply(source); assertThat(source.getSubscription(), nullValue()); PipelineOptions options = PipelineOptionsFactory.create(); @@ -373,7 +370,6 @@ public void noSubscriptionNoSplitGeneratesSubscription() throws Exception { null); assertThat(source.getSubscription(), nullValue()); - TestPipeline.create().apply(source); assertThat(source.getSubscription(), nullValue()); PipelineOptions options = PipelineOptionsFactory.create(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 472399a9d4e2..b8b28eb3b4b1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -146,6 +146,9 @@ public class TextIOTest { private static File tinyZip; private static File largeZip; + @Rule + public TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -224,8 +227,6 @@ private void runTestRead(T[] expected, Coder coder) throws Exception { } } - Pipeline p = TestPipeline.create(); - TextIO.Read.Bound read; if (coder.equals(StringUtf8Coder.of())) { TextIO.Read.Bound readStrings = TextIO.Read.from(filename); @@ -273,7 +274,7 @@ public void testReadNulls() throws Exception { @Test public void testReadNamed() throws Exception { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); assertEquals( "TextIO.Read/Read.out", @@ -330,8 +331,6 @@ private void runTestWrite( Path baseDir = Files.createTempDirectory(tempFolder, "testwrite"); String baseFilename = baseDir.resolve(outputName).toString(); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(Arrays.asList(elems)).withCoder(coder)); TextIO.Write.Bound write; @@ -511,7 +510,6 @@ public void testWriteWithWritableByteChannelFactory() throws Exception { Coder coder = StringUtf8Coder.of(); String outputName = "file.txt"; Path baseDir = Files.createTempDirectory(tempFolder, "testwrite"); - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(LINES2_ARRAY)).withCoder(coder)); @@ -601,11 +599,10 @@ public void testPrimitiveWriteDisplayData() throws IOException { @Test public void testUnsupportedFilePattern() throws IOException { + p.enableAbandonedNodeEnforcement(false); // Windows doesn't like resolving paths with * in them. String filename = tempFolder.resolve("output@5").toString(); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(Arrays.asList(LINES_ARRAY)) .withCoder(StringUtf8Coder.of())); @@ -621,13 +618,13 @@ public void testUnsupportedFilePattern() throws IOException { */ @Test public void testBadWildcardRecursive() throws Exception { - Pipeline pipeline = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); // Check that applying does fail. expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("wildcard"); - pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz")); + p.apply(TextIO.Read.from("gs://bucket/foo**/baz")); } /** Options for testing. */ @@ -641,9 +638,11 @@ public interface RuntimeTestOptions extends PipelineOptions { @Test public void testRuntimeOptionsNotCalledInApply() throws Exception { - Pipeline pipeline = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); + RuntimeTestOptions options = PipelineOptionsFactory.as(RuntimeTestOptions.class); - pipeline + + p .apply(TextIO.Read.from(options.getInput()).withoutValidation()) .apply(TextIO.Write.to(options.getOutput()).withoutValidation()); } @@ -686,12 +685,12 @@ private static void writeToStreamAndClose(String[] lines, OutputStream outputStr * Helper method that runs TextIO.Read.from(filename).withCompressionType(compressionType) * and asserts that the results match the given expected output. */ - private static void assertReadingCompressedFileMatchesExpected( + private void assertReadingCompressedFileMatchesExpected( File file, CompressionType compressionType, String[] expected) { - Pipeline p = TestPipeline.create(); + TextIO.Read.Bound read = TextIO.Read.from(file.getPath()).withCompressionType(compressionType); - PCollection output = p.apply(read); + PCollection output = p.apply("Read_" + file + "_" + compressionType.toString(), read); PAssert.that(output).containsInAnyOrder(expected); p.run(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index 5a7c994d2871..79f4c4b7c2c5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -79,6 +79,7 @@ */ @RunWith(JUnit4.class) public class WriteTest { + @Rule public final TestPipeline p = TestPipeline.create(); @Rule public ExpectedException thrown = ExpectedException.none(); // Static store that can be accessed within the writer @@ -294,7 +295,6 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test public void testWriteUnbounded() { - TestPipeline p = TestPipeline.create(); PCollection unbounded = p.apply(CountingInput.unbounded()) .apply(MapElements.via(new ToStringFn())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java index 1f154d577cce..d6898d5ac0f2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java @@ -40,7 +40,6 @@ import java.util.Random; import javax.xml.bind.annotation.XmlAttribute; import javax.xml.bind.annotation.XmlRootElement; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Source.Reader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -64,6 +63,10 @@ */ @RunWith(JUnit4.class) public class XmlSourceTest { + + @Rule + public TestPipeline p = TestPipeline.create(); + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @@ -566,8 +569,6 @@ public void testReadXMLWithEmptyTags() throws IOException { @Test @Category(NeedsRunner.class) public void testReadXMLSmallPipeline() throws IOException { - Pipeline p = TestPipeline.create(); - File file = tempFolder.newFile("trainXMLSmall"); Files.write(file.toPath(), trainXML.getBytes(StandardCharsets.UTF_8)); @@ -661,7 +662,6 @@ public void testReadXMLLargePipeline() throws IOException { List trains = generateRandomTrainList(100); File file = createRandomTrainXML(fileName, trains); - Pipeline p = TestPipeline.create(); XmlSource source = XmlSource.from(file.toPath().toString()) .withRootElement("trains") @@ -808,8 +808,6 @@ public void testReadXMLFilePattern() throws IOException { generateRandomTrainList(8); createRandomTrainXML("otherfile.xml", trains1); - Pipeline p = TestPipeline.create(); - XmlSource source = XmlSource.from(file.getParent() + "/" + "temp*.xml") .withRootElement("trains") diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java index 5e97eed26c25..4e257f128294 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java @@ -50,7 +50,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -90,6 +89,9 @@ public interface Simple extends PipelineOptions { void setString(String value); } + @Rule + public TestPipeline p = TestPipeline.create(); + @Test public void testPropertySettingAndGetting() throws Exception { ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); @@ -785,7 +787,6 @@ public String toString() { } }; - Pipeline p = TestPipeline.create(); p.getOptions().as(ObjectPipelineOptions.class).setValue(brokenValueType); p.apply(Create.of(1, 2, 3)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index b0c17d8c5aaf..2327459cb9a1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -51,14 +51,16 @@ */ @RunWith(JUnit4.class) public class TransformHierarchyTest { + + @Rule public final TestPipeline pipeline = TestPipeline.create(); @Rule public ExpectedException thrown = ExpectedException.none(); + private TransformHierarchy hierarchy; - private TestPipeline pipeline; + @Before public void setup() { hierarchy = new TransformHierarchy(); - pipeline = TestPipeline.create(); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java index d70aa2f64c77..6a6e0fccb19c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java @@ -55,6 +55,8 @@ */ @RunWith(JUnit4.class) public class TransformTreeTest { + + @Rule public final TestPipeline p = TestPipeline.create(); @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); enum TransformsSeen { @@ -112,11 +114,11 @@ protected Coder getDefaultOutputCoder() { // visits the nodes and verifies that the hierarchy was captured. @Test public void testCompositeCapture() throws Exception { + p.enableAbandonedNodeEnforcement(false); + File inputFile = tmpFolder.newFile(); File outputFile = tmpFolder.newFile(); - Pipeline p = TestPipeline.create(); - p.apply("ReadMyFile", TextIO.Read.from(inputFile.getPath())) .apply(Sample.any(10)) .apply("WriteMyFile", TextIO.Write.to(outputFile.getPath())); @@ -170,18 +172,15 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { @Test(expected = IllegalArgumentException.class) public void testOutputChecking() throws Exception { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); p.apply(new InvalidCompositeTransform()); - p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() {}); } @Test @Category(NeedsRunner.class) public void testMultiGraphSetup() { - Pipeline p = TestPipeline.create(); - PCollection input = p.begin() .apply(Create.of(1, 2, 3)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java index 417147f88fbd..a96e3f8047f5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java @@ -37,6 +37,7 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -45,10 +46,12 @@ /** Tests for {@link GatherAllPanes}. */ @RunWith(JUnit4.class) public class GatherAllPanesTest implements Serializable { + + @Rule public transient TestPipeline p = TestPipeline.create(); + @Test @Category(NeedsRunner.class) public void singlePaneSingleReifiedPane() { - TestPipeline p = TestPipeline.create(); PCollection>>> accumulatedPanes = p.apply(CountingInput.upTo(20000)) .apply( @@ -91,8 +94,6 @@ public Void apply(Iterable>>> input) @Test @Category(NeedsRunner.class) public void multiplePanesMultipleReifiedPane() { - TestPipeline p = TestPipeline.create(); - PCollection someElems = p.apply("someLongs", CountingInput.upTo(20000)); PCollection otherElems = p.apply("otherLongs", CountingInput.upTo(20000)); PCollection>>> accumulatedPanes = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java index be8924f6ab02..1997bbeef337 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java @@ -59,6 +59,10 @@ */ @RunWith(JUnit4.class) public class PAssertTest implements Serializable { + + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -116,8 +120,6 @@ public void registerByteSizeObserver( @Test @Category(RunnableOnService.class) public void testContainsInAnyOrderNotSerializable() throws Exception { - Pipeline pipeline = TestPipeline.create(); - PCollection pcollection = pipeline .apply(Create.of( new NotSerializableObject(), @@ -139,8 +141,6 @@ public void testContainsInAnyOrderNotSerializable() throws Exception { @Test @Category(RunnableOnService.class) public void testSerializablePredicate() throws Exception { - Pipeline pipeline = TestPipeline.create(); - PCollection pcollection = pipeline .apply(Create.of( new NotSerializableObject(), @@ -166,8 +166,6 @@ public Void apply(Iterable contents) { @Test @Category(RunnableOnService.class) public void testWindowedSerializablePredicate() throws Exception { - Pipeline pipeline = TestPipeline.create(); - PCollection pcollection = pipeline .apply(Create.timestamped( TimestampedValue.of(new NotSerializableObject(), new Instant(250L)), @@ -207,7 +205,6 @@ public void testPAssertEqualsSingletonUnsupported() throws Exception { thrown.expect(UnsupportedOperationException.class); thrown.expectMessage("isEqualTo"); - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(42)); PAssert.thatSingleton(pcollection).equals(42); } @@ -222,7 +219,6 @@ public void testPAssertEqualsIterableUnsupported() throws Exception { thrown.expect(UnsupportedOperationException.class); thrown.expectMessage("containsInAnyOrder"); - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(42)); PAssert.that(pcollection).equals(42); } @@ -237,7 +233,6 @@ public void testPAssertHashCodeSingletonUnsupported() throws Exception { thrown.expect(UnsupportedOperationException.class); thrown.expectMessage(".hashCode() is not supported."); - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(42)); PAssert.thatSingleton(pcollection).hashCode(); } @@ -252,7 +247,6 @@ public void testPAssertHashCodeIterableUnsupported() throws Exception { thrown.expect(UnsupportedOperationException.class); thrown.expectMessage(".hashCode() is not supported."); - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(42)); PAssert.that(pcollection).hashCode(); } @@ -263,7 +257,6 @@ public void testPAssertHashCodeIterableUnsupported() throws Exception { @Test @Category(RunnableOnService.class) public void testIsEqualTo() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(43)); PAssert.thatSingleton(pcollection).isEqualTo(43); pipeline.run(); @@ -275,7 +268,6 @@ public void testIsEqualTo() throws Exception { @Test @Category(RunnableOnService.class) public void testWindowedIsEqualTo() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.timestamped(TimestampedValue.of(43, new Instant(250L)), TimestampedValue.of(22, new Instant(-250L)))) @@ -295,7 +287,6 @@ public void testWindowedIsEqualTo() throws Exception { @Test @Category(RunnableOnService.class) public void testNotEqualTo() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(43)); PAssert.thatSingleton(pcollection).notEqualTo(42); pipeline.run(); @@ -307,7 +298,6 @@ public void testNotEqualTo() throws Exception { @Test @Category(RunnableOnService.class) public void testContainsInAnyOrder() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(1, 2, 3, 4)); PAssert.that(pcollection).containsInAnyOrder(2, 1, 4, 3); pipeline.run(); @@ -319,7 +309,6 @@ public void testContainsInAnyOrder() throws Exception { @Test @Category(RunnableOnService.class) public void testGlobalWindowContainsInAnyOrder() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.of(1, 2, 3, 4)); PAssert.that(pcollection).inWindow(GlobalWindow.INSTANCE).containsInAnyOrder(2, 1, 4, 3); pipeline.run(); @@ -331,7 +320,6 @@ public void testGlobalWindowContainsInAnyOrder() throws Exception { @Test @Category(RunnableOnService.class) public void testWindowedContainsInAnyOrder() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection pcollection = pipeline.apply(Create.timestamped(TimestampedValue.of(1, new Instant(100L)), TimestampedValue.of(2, new Instant(200L)), @@ -361,13 +349,12 @@ public void testWindowedContainsInAnyOrder() throws Exception { @Test @Category(RunnableOnService.class) public void testEmpty() { - Pipeline p = TestPipeline.create(); PCollection vals = - p.apply(Create.of().withCoder(VarLongCoder.of())); + pipeline.apply(Create.of().withCoder(VarLongCoder.of())); PAssert.that(vals).empty(); - p.run(); + pipeline.run(); } /** @@ -376,8 +363,6 @@ public void testEmpty() { @Test @Category(RunnableOnService.class) public void testContainsInAnyOrderFalse() throws Exception { - Pipeline pipeline = TestPipeline.create(); - PCollection pcollection = pipeline .apply(Create.of(1, 2, 3, 4)); @@ -399,11 +384,10 @@ public void testContainsInAnyOrderFalse() throws Exception { @Test @Category(RunnableOnService.class) public void testEmptyFalse() throws Exception { - Pipeline p = TestPipeline.create(); - PCollection vals = p.apply(CountingInput.upTo(5L)); + PCollection vals = pipeline.apply(CountingInput.upTo(5L)); PAssert.that(vals).empty(); - Throwable thrown = runExpectingAssertionFailure(p); + Throwable thrown = runExpectingAssertionFailure(pipeline); assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order")); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index a1b4e4a8b3ed..64aeca3cfe21 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -65,6 +65,7 @@ */ @RunWith(JUnit4.class) public class TestStreamTest implements Serializable { + @Rule public transient TestPipeline p = TestPipeline.create(); @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test @@ -85,7 +86,6 @@ public void testLateDataAccumulating() { TimestampedValue.of(-3, instant)) .advanceWatermarkToInfinity(); - TestPipeline p = TestPipeline.create(); PCollection windowed = p .apply(source) .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5))).triggering( @@ -146,7 +146,6 @@ public void testProcessingTimeTrigger() { .advanceProcessingTime(Duration.standardMinutes(6)) .advanceWatermarkToInfinity(); - TestPipeline p = TestPipeline.create(); PCollection sum = p.apply(source) .apply(Window.triggering(AfterWatermark.pastEndOfWindow() .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() @@ -175,7 +174,6 @@ public void testDiscardingMode() { TimestampedValue.of("alsoFinalLatePane", new Instant(250))) .advanceWatermarkToInfinity(); - TestPipeline p = TestPipeline.create(); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); Duration allowedLateness = Duration.millis(5000L); PCollection values = @@ -220,7 +218,6 @@ public void testFirstElementLate() { .addElements(TimestampedValue.of("onTime", new Instant(100))) .advanceWatermarkToInfinity(); - TestPipeline p = TestPipeline.create(); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); Duration allowedLateness = Duration.millis(5000L); PCollection values = p.apply(stream) @@ -249,7 +246,6 @@ public void testElementsAtAlmostPositiveInfinity() { TimestampedValue.of("bar", endOfGlobalWindow)) .advanceWatermarkToInfinity(); - TestPipeline p = TestPipeline.create(); FixedWindows windows = FixedWindows.of(Duration.standardHours(6)); PCollection windowedValues = p.apply(stream) .apply(Window.into(windows)) @@ -274,7 +270,6 @@ public void testMultipleStreams() { TestStream other = TestStream.create(VarIntCoder.of()).addElements(1, 2, 3, 4).advanceWatermarkToInfinity(); - TestPipeline p = TestPipeline.create(); PCollection createStrings = p.apply("CreateStrings", stream) .apply("WindowStrings", diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java index ab1394697c4f..cd7898bc95a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java @@ -42,6 +42,7 @@ import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -63,6 +64,9 @@ public class ApproximateQuantilesTest { KV.of("b", 100) ); + @Rule + public TestPipeline p = TestPipeline.create(); + public PCollection> createInputTable(Pipeline p) { return p.apply(Create.of(TABLE).withCoder( KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -71,8 +75,6 @@ public PCollection> createInputTable(Pipeline p) { @Test @Category(NeedsRunner.class) public void testQuantilesGlobally() { - TestPipeline p = TestPipeline.create(); - PCollection input = intRangeCollection(p, 101); PCollection> quantiles = input.apply(ApproximateQuantiles.globally(5)); @@ -85,8 +87,6 @@ public void testQuantilesGlobally() { @Test @Category(NeedsRunner.class) public void testQuantilesGobally_comparable() { - TestPipeline p = TestPipeline.create(); - PCollection input = intRangeCollection(p, 101); PCollection> quantiles = input.apply( @@ -100,8 +100,6 @@ public void testQuantilesGobally_comparable() { @Test @Category(NeedsRunner.class) public void testQuantilesPerKey() { - Pipeline p = TestPipeline.create(); - PCollection> input = createInputTable(p); PCollection>> quantiles = input.apply( ApproximateQuantiles.perKey(2)); @@ -117,8 +115,6 @@ public void testQuantilesPerKey() { @Test @Category(NeedsRunner.class) public void testQuantilesPerKey_reversed() { - Pipeline p = TestPipeline.create(); - PCollection> input = createInputTable(p); PCollection>> quantiles = input.apply( ApproximateQuantiles.perKey( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java index b63c73d59885..3afc759d5d43 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -52,6 +53,9 @@ public class ApproximateUniqueTest implements Serializable { // implements Serializable just to make it easy to use anonymous inner DoFn subclasses + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test public void testEstimationErrorToSampleSize() { assertEquals(40000, ApproximateUnique.sampleSizeFromEstimationError(0.01)); @@ -67,8 +71,6 @@ public void testEstimationErrorToSampleSize() { @Test @Category(RunnableOnService.class) public void testApproximateUniqueWithSmallInput() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply( Create.of(Arrays.asList(1, 2, 3, 3))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java index 88625316d37a..cdd4707870f8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java @@ -28,7 +28,6 @@ import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -62,6 +61,7 @@ */ @RunWith(JUnit4.class) public class CombineFnsTest { + @Rule public final TestPipeline p = TestPipeline.create(); @Rule public ExpectedException expectedException = ExpectedException.none(); @Test @@ -123,7 +123,6 @@ public void testDuplicatedTagsWithContextKeyed() { @Test @Category(RunnableOnService.class) public void testComposedCombine() { - Pipeline p = TestPipeline.create(); p.getCoderRegistry().registerCoder(UserString.class, UserStringCoder.of()); PCollection>> perKeyInput = p.apply( @@ -178,7 +177,6 @@ public void testComposedCombine() { @Test @Category(RunnableOnService.class) public void testComposedCombineWithContext() { - Pipeline p = TestPipeline.create(); p.getCoderRegistry().registerCoder(UserString.class, UserStringCoder.of()); PCollectionView view = p @@ -240,7 +238,6 @@ public void testComposedCombineWithContext() { @Test @Category(RunnableOnService.class) public void testComposedCombineNullValues() { - Pipeline p = TestPipeline.create(); p.getCoderRegistry().registerCoder(UserString.class, NullableCoder.of(UserStringCoder.of())); p.getCoderRegistry().registerCoder(String.class, NullableCoder.of(StringUtf8Coder.of())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 671f00e5da95..0ac950264d51 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -82,6 +82,7 @@ import org.apache.beam.sdk.values.POutput; import org.hamcrest.Matchers; import org.joda.time.Duration; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -108,6 +109,9 @@ public class CombineTest implements Serializable { @Mock private DoFn.ProcessContext processContext; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + PCollection> createInput(Pipeline p, List> table) { return p.apply(Create.of(table).withCoder( @@ -117,7 +121,6 @@ PCollection> createInput(Pipeline p, private void runTestSimpleCombine(List> table, int globalSum, List> perKeyCombines) { - Pipeline pipeline = TestPipeline.create(); PCollection> input = createInput(pipeline, table); PCollection sum = input @@ -138,7 +141,6 @@ private void runTestSimpleCombineWithContext(List> table, int globalSum, List> perKeyCombines, String[] globallyCombines) { - Pipeline pipeline = TestPipeline.create(); PCollection> perKeyInput = createInput(pipeline, table); PCollection globallyInput = perKeyInput.apply(Values.create()); @@ -197,7 +199,6 @@ public void testSimpleCombineEmpty() { private void runTestBasicCombine(List> table, Set globalUnique, List>> perKeyUnique) { - Pipeline pipeline = TestPipeline.create(); pipeline.getCoderRegistry().registerCoder(Set.class, SetCoder.class); PCollection> input = createInput(pipeline, table); @@ -233,7 +234,6 @@ public void testBasicCombineEmpty() { private void runTestAccumulatingCombine(List> table, Double globalMean, List> perKeyMeans) { - Pipeline pipeline = TestPipeline.create(); PCollection> input = createInput(pipeline, table); PCollection mean = input @@ -253,8 +253,6 @@ private void runTestAccumulatingCombine(List> table, @Test @Category(RunnableOnService.class) public void testFixedWindowsCombine() { - Pipeline pipeline = TestPipeline.create(); - PCollection> input = pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 1L, 6L, 7L, 8L)) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) @@ -279,8 +277,6 @@ public void testFixedWindowsCombine() { @Test @Category(RunnableOnService.class) public void testFixedWindowsCombineWithContext() { - Pipeline pipeline = TestPipeline.create(); - PCollection> perKeyInput = pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 1L, 6L, 7L, 8L)) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) @@ -316,8 +312,6 @@ public void testFixedWindowsCombineWithContext() { @Test @Category(RunnableOnService.class) public void testSlidingWindowsCombineWithContext() { - Pipeline pipeline = TestPipeline.create(); - PCollection> perKeyInput = pipeline.apply(Create.timestamped(TABLE, Arrays.asList(2L, 3L, 8L, 9L, 10L)) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) @@ -365,7 +359,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testGlobalCombineWithDefaultsAndTriggers() { - Pipeline pipeline = TestPipeline.create(); PCollection input = pipeline.apply(Create.of(1, 1)); PCollection output = input @@ -392,8 +385,6 @@ public Void apply(Iterable input) { @Test @Category(RunnableOnService.class) public void testSessionsCombine() { - Pipeline pipeline = TestPipeline.create(); - PCollection> input = pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 4L, 7L, 10L, 16L)) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) @@ -417,8 +408,6 @@ public void testSessionsCombine() { @Test @Category(RunnableOnService.class) public void testSessionsCombineWithContext() { - Pipeline pipeline = TestPipeline.create(); - PCollection> perKeyInput = pipeline.apply(Create.timestamped(TABLE, Arrays.asList(0L, 4L, 7L, 10L, 16L)) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -459,8 +448,6 @@ public void testSessionsCombineWithContext() { @Test @Category(RunnableOnService.class) public void testWindowedCombineEmpty() { - Pipeline pipeline = TestPipeline.create(); - PCollection mean = pipeline .apply(Create.of().withCoder(BigEndianIntegerCoder.of())) .apply(Window.into(FixedWindows.of(Duration.millis(1)))) @@ -517,7 +504,6 @@ public Integer apply(String input) { @Test @Category(RunnableOnService.class) public void testHotKeyCombining() { - Pipeline pipeline = TestPipeline.create(); PCollection> input = copy(createInput(pipeline, TABLE), 10); KeyedCombineFn mean = @@ -552,7 +538,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testHotKeyCombiningWithAccumulationMode() { - Pipeline pipeline = TestPipeline.create(); PCollection input = pipeline.apply(Create.of(1, 2, 3, 4, 5)); PCollection output = input @@ -577,7 +562,6 @@ public Void apply(Iterable input) { @Test @Category(NeedsRunner.class) public void testBinaryCombineFn() { - Pipeline pipeline = TestPipeline.create(); PCollection> input = copy(createInput(pipeline, TABLE), 2); PCollection> intProduct = input .apply("IntProduct", Combine.perKey(new TestProdInt())); @@ -632,7 +616,6 @@ public Integer apply(Integer left, Integer right) { @Test @Category(RunnableOnService.class) public void testCombineGloballyAsSingletonView() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView view = pipeline .apply("CreateEmptySideInput", Create.of().withCoder(BigEndianIntegerCoder.of())) .apply(Sum.integersGlobally().asSingletonView()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java index 7f77ae7f64c3..eafb12d1fcd8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CountTest.java @@ -22,13 +22,13 @@ import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; 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.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -45,12 +45,13 @@ public class CountTest { static final List WORDS = Arrays.asList(WORDS_ARRAY); + @Rule + public TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) @SuppressWarnings("unchecked") public void testCountPerElementBasic() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(WORDS)); PCollection> output = @@ -71,8 +72,6 @@ public void testCountPerElementBasic() { @Category(RunnableOnService.class) @SuppressWarnings("unchecked") public void testCountPerElementEmpty() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(NO_LINES).withCoder(StringUtf8Coder.of())); PCollection> output = @@ -85,8 +84,6 @@ public void testCountPerElementEmpty() { @Test @Category(RunnableOnService.class) public void testCountGloballyBasic() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(WORDS)); PCollection output = @@ -100,8 +97,6 @@ public void testCountGloballyBasic() { @Test @Category(RunnableOnService.class) public void testCountGloballyEmpty() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(NO_LINES).withCoder(StringUtf8Coder.of())); PCollection output = 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 2a89a1837df7..3bc0a65df2d5 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 @@ -38,7 +38,6 @@ import java.util.Collections; import java.util.List; import java.util.Random; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -76,12 +75,12 @@ @SuppressWarnings("unchecked") public class CreateTest { @Rule public final ExpectedException thrown = ExpectedException.none(); + @Rule public final TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testCreate() { - Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of(LINES)); @@ -93,8 +92,6 @@ public void testCreate() { @Test @Category(RunnableOnService.class) public void testCreateEmpty() { - Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of(NO_LINES) .withCoder(StringUtf8Coder.of())); @@ -106,7 +103,7 @@ public void testCreateEmpty() { @Test public void testCreateEmptyInfersCoder() { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); PCollection output = p.apply(Create.of()); @@ -126,8 +123,6 @@ public void testPolymorphicType() throws Exception { thrown.expectMessage( Matchers.containsString("Unable to infer a coder")); - Pipeline p = TestPipeline.create(); - // Create won't infer a default coder in this case. p.apply(Create.of(new Record(), new Record2())); @@ -137,8 +132,6 @@ public void testPolymorphicType() throws Exception { @Test @Category(RunnableOnService.class) public void testCreateWithNullsAndValues() throws Exception { - Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of(null, "test1", null, "test2", null) .withCoder(SerializableCoder.of(String.class))); @@ -150,8 +143,6 @@ public void testCreateWithNullsAndValues() throws Exception { @Test @Category(NeedsRunner.class) public void testCreateParameterizedType() throws Exception { - Pipeline p = TestPipeline.create(); - PCollection> output = p.apply(Create.of( TimestampedValue.of("a", new Instant(0)), @@ -216,7 +207,6 @@ public void testCreateWithUnserializableElements() throws Exception { Create.Values create = Create.of(elements).withCoder(new UnserializableRecord.UnserializableRecordCoder()); - TestPipeline p = TestPipeline.create(); PAssert.that(p.apply(create)) .containsInAnyOrder( new UnserializableRecord("foo"), @@ -235,8 +225,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testCreateTimestamped() { - Pipeline p = TestPipeline.create(); - List> data = Arrays.asList( TimestampedValue.of("a", new Instant(1L)), TimestampedValue.of("b", new Instant(2L)), @@ -254,8 +242,6 @@ public void testCreateTimestamped() { @Test @Category(RunnableOnService.class) public void testCreateTimestampedEmpty() { - Pipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.timestamped(new ArrayList>()) .withCoder(StringUtf8Coder.of())); @@ -266,7 +252,7 @@ public void testCreateTimestampedEmpty() { @Test public void testCreateTimestampedEmptyInfersCoder() { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); PCollection output = p .apply(Create.timestamped()); @@ -280,8 +266,6 @@ public void testCreateTimestampedPolymorphicType() throws Exception { thrown.expectMessage( Matchers.containsString("Unable to infer a coder")); - Pipeline p = TestPipeline.create(); - // Create won't infer a default coder in this case. PCollection c = p.apply(Create.timestamped( TimestampedValue.of(new Record(), new Instant(0)), @@ -295,7 +279,6 @@ public void testCreateTimestampedPolymorphicType() throws Exception { @Test @Category(RunnableOnService.class) public void testCreateWithVoidType() throws Exception { - Pipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of((Void) null, (Void) null)); PAssert.that(output).containsInAnyOrder((Void) null, (Void) null); p.run(); @@ -304,8 +287,6 @@ public void testCreateWithVoidType() throws Exception { @Test @Category(RunnableOnService.class) public void testCreateWithKVVoidType() throws Exception { - Pipeline p = TestPipeline.create(); - PCollection> output = p.apply(Create.of( KV.of((Void) null, (Void) null), KV.of((Void) null, (Void) null))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java index 257b364e9147..b3b39250f794 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java @@ -24,13 +24,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; 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.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -41,6 +41,10 @@ */ @RunWith(JUnit4.class) public class DistinctTest { + + @Rule + public final TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testDistinct() { @@ -53,8 +57,6 @@ public void testDistinct() { "k2", "k3"); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(strings) .withCoder(StringUtf8Coder.of())); @@ -72,8 +74,6 @@ public void testDistinct() { public void testDistinctEmpty() { List strings = Arrays.asList(); - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(strings) .withCoder(StringUtf8Coder.of())); @@ -115,8 +115,6 @@ public void testDistinctWithRepresentativeValue() { KV.of("k1", "v2"), KV.of("k2", "v1")); - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(strings)); PCollection> output = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java index e5f5cb6547f5..19b7c51e54b5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTest.java @@ -40,6 +40,9 @@ /** Tests for {@link DoFn}. */ @RunWith(JUnit4.class) public class DoFnTest implements Serializable { + + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -200,7 +203,6 @@ public void processElement(ProcessContext c) {} * Initialize a test pipeline with the specified {@link DoFn}. */ private TestPipeline createTestPipeline(DoFn fn) { - TestPipeline pipeline = TestPipeline.create(); pipeline.apply(Create.of((InputT) null)) .apply(ParDo.of(fn)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index 2dafa27d99d7..3859c9f4be4c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -52,6 +52,8 @@ */ @RunWith(JUnit4.class) public class DoFnTesterTest { + + @Rule public final TestPipeline p = TestPipeline.create(); @Rule public ExpectedException thrown = ExpectedException.none(); @Test @@ -324,7 +326,7 @@ public void peekValuesInWindow() throws Exception { public void fnWithSideInputDefault() throws Exception { final PCollectionView value = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of()); try (DoFnTester tester = DoFnTester.of(new SideInputDoFn(value))) { tester.processElement(1); @@ -339,7 +341,7 @@ public void fnWithSideInputDefault() throws Exception { public void fnWithSideInputExplicit() throws Exception { final PCollectionView value = PCollectionViews.singletonView( - TestPipeline.create(), WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of()); + p, WindowingStrategy.globalDefault(), true, 0, VarIntCoder.of()); try (DoFnTester tester = DoFnTester.of(new SideInputDoFn(value))) { tester.setSideInput(value, GlobalWindow.INSTANCE, -2); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java index 5221f75b956b..81e1d022e774 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -57,11 +58,12 @@ public Boolean apply(Integer elem) { } } + @Rule + public final TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testIdentityFilterByPredicate() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(591, 11789, 1257, 24578, 24799, 307)) .apply(Filter.by(new TrivialFn(true))); @@ -73,8 +75,6 @@ public void testIdentityFilterByPredicate() { @Test @Category(RunnableOnService.class) public void testNoFilterByPredicate() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(1, 2, 4, 5)) .apply(Filter.by(new TrivialFn(false))); @@ -86,8 +86,6 @@ public void testNoFilterByPredicate() { @Test @Category(RunnableOnService.class) public void testFilterByPredicate() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) .apply(Filter.by(new EvenFn())); @@ -99,8 +97,6 @@ public void testFilterByPredicate() { @Test @Category(RunnableOnService.class) public void testFilterLessThan() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) .apply(Filter.lessThan(4)); @@ -112,8 +108,6 @@ public void testFilterLessThan() { @Test @Category(RunnableOnService.class) public void testFilterGreaterThan() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) .apply(Filter.greaterThan(4)); @@ -125,8 +119,6 @@ public void testFilterGreaterThan() { @Test @Category(RunnableOnService.class) public void testFilterLessThanEq() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) .apply(Filter.lessThanEq(4)); @@ -138,8 +130,6 @@ public void testFilterLessThanEq() { @Test @Category(RunnableOnService.class) public void testFilterGreaterThanEq() { - TestPipeline p = TestPipeline.create(); - PCollection output = p .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) .apply(Filter.greaterThanEq(4)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java index bb2877ec5eb9..b24071ec72cf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java @@ -27,7 +27,6 @@ import java.util.Collections; import java.util.List; import java.util.Set; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -48,6 +47,9 @@ @RunWith(JUnit4.class) public class FlatMapElementsTest implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -57,7 +59,6 @@ public class FlatMapElementsTest implements Serializable { @Test @Category(NeedsRunner.class) public void testFlatMapBasic() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3)) @@ -82,7 +83,6 @@ public List apply(Integer input) { @Test @Category(NeedsRunner.class) public void testFlatMapFnOutputTypeDescriptor() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of("hello")) .apply(FlatMapElements.via(new SimpleFunction>() { @@ -117,7 +117,8 @@ public Iterable apply(T input) { */ @Test public void testPolymorphicSimpleFunction() throws Exception { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + PCollection output = pipeline .apply(Create.of(1, 2, 3)) @@ -168,7 +169,6 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test @Category(NeedsRunner.class) public void testVoidValues() throws Exception { - Pipeline pipeline = TestPipeline.create(); pipeline .apply(Create.of("hello")) .apply(WithKeys.of("k")) 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 d4686a41f8d5..48251bc0247c 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 @@ -64,6 +64,9 @@ @RunWith(JUnit4.class) public class FlattenTest implements Serializable { + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -74,8 +77,6 @@ private static class ClassWithoutCoder { } @Test @Category(RunnableOnService.class) public void testFlattenPCollectionList() { - Pipeline p = TestPipeline.create(); - List> inputs = Arrays.asList( LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES); @@ -90,8 +91,6 @@ public void testFlattenPCollectionList() { @Test @Category(RunnableOnService.class) public void testFlattenPCollectionListThenParDo() { - Pipeline p = TestPipeline.create(); - List> inputs = Arrays.asList( LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES); @@ -107,8 +106,6 @@ public void testFlattenPCollectionListThenParDo() { @Test @Category(RunnableOnService.class) public void testFlattenPCollectionListEmpty() { - Pipeline p = TestPipeline.create(); - PCollection output = PCollectionList.empty(p) .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()); @@ -120,8 +117,6 @@ public void testFlattenPCollectionListEmpty() { @Test @Category(RunnableOnService.class) public void testFlattenInputMultipleCopies() { - Pipeline p = TestPipeline.create(); - int count = 5; PCollection longs = p.apply("mkLines", CountingInput.upTo(count)); PCollection biggerLongs = @@ -154,8 +149,6 @@ public Long apply(Long input) { @Test @Category(RunnableOnService.class) public void testEmptyFlattenAsSideInput() { - Pipeline p = TestPipeline.create(); - final PCollectionView> view = PCollectionList.empty(p) .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()) @@ -179,9 +172,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testFlattenPCollectionListEmptyThenParDo() { - - Pipeline p = TestPipeline.create(); - PCollection output = PCollectionList.empty(p) .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()) @@ -198,8 +188,6 @@ public void testFlattenNoListsNoCoder() { thrown.expect(IllegalStateException.class); thrown.expectMessage("cannot provide a Coder for empty"); - Pipeline p = TestPipeline.create(); - PCollectionList.empty(p) .apply(Flatten.pCollections()); @@ -211,8 +199,6 @@ public void testFlattenNoListsNoCoder() { @Test @Category(RunnableOnService.class) public void testFlattenIterables() { - Pipeline p = TestPipeline.create(); - PCollection> input = p .apply(Create.>of(LINES) .withCoder(IterableCoder.of(StringUtf8Coder.of()))); @@ -229,8 +215,6 @@ public void testFlattenIterables() { @Test @Category(RunnableOnService.class) public void testFlattenIterablesLists() { - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.>of(LINES).withCoder(ListCoder.of(StringUtf8Coder.of()))); @@ -244,8 +228,6 @@ public void testFlattenIterablesLists() { @Test @Category(RunnableOnService.class) public void testFlattenIterablesSets() { - Pipeline p = TestPipeline.create(); - Set linesSet = ImmutableSet.copyOf(LINES); PCollection> input = @@ -261,9 +243,6 @@ public void testFlattenIterablesSets() { @Test @Category(RunnableOnService.class) public void testFlattenIterablesCollections() { - - Pipeline p = TestPipeline.create(); - Set linesSet = ImmutableSet.copyOf(LINES); PCollection> input = @@ -280,8 +259,6 @@ public void testFlattenIterablesCollections() { @Test @Category(RunnableOnService.class) public void testFlattenIterablesEmpty() { - Pipeline p = TestPipeline.create(); - PCollection> input = p .apply(Create.>of(NO_LINES) .withCoder(IterableCoder.of(StringUtf8Coder.of()))); @@ -300,8 +277,6 @@ public void testFlattenIterablesEmpty() { @Test @Category(NeedsRunner.class) public void testEqualWindowFnPropagation() { - Pipeline p = TestPipeline.create(); - PCollection input1 = p.apply("CreateInput1", Create.of("Input1")) .apply("Window1", Window.into(FixedWindows.of(Duration.standardMinutes(1)))); @@ -322,8 +297,6 @@ public void testEqualWindowFnPropagation() { @Test @Category(NeedsRunner.class) public void testCompatibleWindowFnPropagation() { - Pipeline p = TestPipeline.create(); - PCollection input1 = p.apply("CreateInput1", Create.of("Input1")) .apply("Window1", @@ -345,7 +318,7 @@ public void testCompatibleWindowFnPropagation() { @Test public void testIncompatibleWindowFnPropagationFailure() { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); PCollection input1 = p.apply("CreateInput1", Create.of("Input1")) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index ebde11052cd3..f4bec3a2d37c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -39,7 +39,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -80,6 +79,9 @@ @SuppressWarnings({"rawtypes", "unchecked"}) public class GroupByKeyTest { + @Rule + public final TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); @@ -95,8 +97,6 @@ public void testGroupByKey() { KV.of("k2", -33), KV.of("k3", 0)); - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(ungroupedPairs) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -137,8 +137,6 @@ public void testGroupByKeyAndWindows() { KV.of("k2", -33), // window [5, 10) KV.of("k3", 0)); // window [5, 10) - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.timestamped(ungroupedPairs, Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L)) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -174,8 +172,6 @@ public Void apply(Iterable>> actual) { public void testGroupByKeyEmpty() { List> ungroupedPairs = Arrays.asList(); - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(ungroupedPairs) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -193,8 +189,6 @@ public void testGroupByKeyNonDeterministic() throws Exception { List, Integer>> ungroupedPairs = Arrays.asList(); - Pipeline p = TestPipeline.create(); - PCollection, Integer>> input = p.apply(Create.of(ungroupedPairs) .withCoder( @@ -209,7 +203,6 @@ public void testGroupByKeyNonDeterministic() throws Exception { @Test @Category(NeedsRunner.class) public void testIdentityWindowFnPropagation() { - Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -230,7 +223,6 @@ public void testIdentityWindowFnPropagation() { @Test @Category(NeedsRunner.class) public void testWindowFnInvalidation() { - Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -255,7 +247,6 @@ public void testWindowFnInvalidation() { @Test public void testInvalidWindowsDirect() { - Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -275,7 +266,6 @@ public void testInvalidWindowsDirect() { @Test @Category(NeedsRunner.class) public void testRemerge() { - Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -300,7 +290,6 @@ public void testRemerge() { @Test public void testGroupByKeyDirectUnbounded() { - Pipeline p = TestPipeline.create(); PCollection> input = p.apply( @@ -331,9 +320,8 @@ public PCollection> expand(PBegin input) { @Test @Category(RunnableOnService.class) public void testOutputTimeFnEarliest() { - Pipeline pipeline = TestPipeline.create(); - pipeline.apply( + p.apply( Create.timestamped( TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) @@ -342,7 +330,7 @@ public void testOutputTimeFnEarliest() { .apply(GroupByKey.create()) .apply(ParDo.of(new AssertTimestamp(new Instant(0)))); - pipeline.run(); + p.run(); } @@ -353,9 +341,7 @@ public void testOutputTimeFnEarliest() { @Test @Category(RunnableOnService.class) public void testOutputTimeFnLatest() { - Pipeline pipeline = TestPipeline.create(); - - pipeline.apply( + p.apply( Create.timestamped( TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) @@ -364,7 +350,7 @@ public void testOutputTimeFnLatest() { .apply(GroupByKey.create()) .apply(ParDo.of(new AssertTimestamp(new Instant(10)))); - pipeline.run(); + p.run(); } private static class AssertTimestamp extends DoFn, Void> { @@ -408,8 +394,6 @@ public void testGroupByKeyWithBadEqualsHashCode() throws Exception { final int numValues = 10; final int numKeys = 5; - Pipeline p = TestPipeline.create(); - p.getCoderRegistry().registerCoder(BadEqualityKey.class, DeterministicKeyCoder.class); // construct input data diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java index fce5b2f205c3..2a19802eea66 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KeysTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.transforms; import java.util.Arrays; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -27,6 +26,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -50,11 +50,12 @@ public class KeysTest { static final KV[] EMPTY_TABLE = new KV[] { }; + @Rule + public final TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testKeys() { - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(Arrays.asList(TABLE)).withCoder( KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -69,8 +70,6 @@ public void testKeys() { @Test @Category(RunnableOnService.class) public void testKeysEmpty() { - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(Arrays.asList(EMPTY_TABLE)).withCoder( KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java index 359819875415..24186ed288f8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.transforms; import java.util.Arrays; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -27,6 +26,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -50,11 +50,12 @@ public class KvSwapTest { static final KV[] EMPTY_TABLE = new KV[] { }; + @Rule + public final TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testKvSwap() { - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(Arrays.asList(TABLE)).withCoder( KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); @@ -75,8 +76,6 @@ public void testKvSwap() { @Test @Category(RunnableOnService.class) public void testKvSwapEmpty() { - Pipeline p = TestPipeline.create(); - PCollection> input = p.apply(Create.of(Arrays.asList(EMPTY_TABLE)).withCoder( KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java index ce9ae376ad34..f71b8138b2a8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java @@ -51,12 +51,13 @@ */ @RunWith(JUnit4.class) public class LatestTest implements Serializable { + + @Rule public final transient TestPipeline p = TestPipeline.create(); @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test @Category(NeedsRunner.class) public void testGloballyEventTimestamp() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.timestamped( TimestampedValue.of("foo", new Instant(100)), @@ -71,7 +72,8 @@ public void testGloballyEventTimestamp() { @Test public void testGloballyOutputCoder() { - TestPipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); + BigEndianLongCoder inputCoder = BigEndianLongCoder.of(); PCollection output = @@ -86,7 +88,6 @@ public void testGloballyOutputCoder() { @Test @Category(NeedsRunner.class) public void testGloballyEmptyCollection() { - TestPipeline p = TestPipeline.create(); PCollection emptyInput = p.apply(Create.of() // Explicitly set coder such that then runner enforces encodability. .withCoder(VarLongCoder.of())); @@ -99,7 +100,6 @@ public void testGloballyEmptyCollection() { @Test @Category(NeedsRunner.class) public void testPerKeyEventTimestamp() { - TestPipeline p = TestPipeline.create(); PCollection> output = p.apply(Create.timestamped( TimestampedValue.of(KV.of("A", "foo"), new Instant(100)), @@ -114,7 +114,8 @@ public void testPerKeyEventTimestamp() { @Test public void testPerKeyOutputCoder() { - TestPipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); + KvCoder inputCoder = KvCoder.of( AvroCoder.of(String.class), AvroCoder.of(Long.class)); @@ -128,7 +129,6 @@ public void testPerKeyOutputCoder() { @Test @Category(NeedsRunner.class) public void testPerKeyEmptyCollection() { - TestPipeline p = TestPipeline.create(); PCollection> output = p.apply(Create.>of().withCoder(KvCoder.of( StringUtf8Coder.of(), StringUtf8Coder.of()))) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java index ac3444b3ae89..47d0b87f6580 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java @@ -24,7 +24,6 @@ import java.io.Serializable; import java.util.Set; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -47,6 +46,9 @@ @RunWith(JUnit4.class) public class MapElementsTest implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -79,7 +81,6 @@ public KV apply(T input) { @Test @Category(NeedsRunner.class) public void testMapBasic() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3)) .apply(MapElements.via(new SimpleFunction() { @@ -98,7 +99,8 @@ public Integer apply(Integer input) { */ @Test public void testPolymorphicSimpleFunction() throws Exception { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + PCollection output = pipeline .apply(Create.of(1, 2, 3)) @@ -120,7 +122,8 @@ public Integer apply(Integer input) { */ @Test public void testNestedPolymorphicSimpleFunction() throws Exception { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + PCollection output = pipeline .apply(Create.of(1, 2, 3)) @@ -149,7 +152,6 @@ public Integer apply(KV input) { @Test @Category(NeedsRunner.class) public void testMapBasicSerializableFunction() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3)) .apply(MapElements.via(new SerializableFunction() { @@ -170,7 +172,6 @@ public Integer apply(Integer input) { @Test @Category(NeedsRunner.class) public void testSimpleFunctionOutputTypeDescriptor() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of("hello")) .apply(MapElements.via(new SimpleFunction() { @@ -191,7 +192,6 @@ public String apply(String input) { @Test @Category(NeedsRunner.class) public void testVoidValues() throws Exception { - Pipeline pipeline = TestPipeline.create(); pipeline .apply(Create.of("hello")) .apply(WithKeys.of("k")) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java index 9bc8a6480555..2c3a73542cf6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -43,10 +44,13 @@ */ @RunWith(JUnit4.class) public class ParDoLifecycleTest implements Serializable { + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testOldFnCallSequence() { - TestPipeline p = TestPipeline.create(); PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) @@ -58,7 +62,6 @@ public void testOldFnCallSequence() { @Test @Category(RunnableOnService.class) public void testOldFnCallSequenceMulti() { - TestPipeline p = TestPipeline.create(); PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) @@ -127,7 +130,6 @@ public void teardown() { @Test @Category(RunnableOnService.class) public void testFnCallSequence() { - TestPipeline p = TestPipeline.create(); PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) @@ -139,7 +141,6 @@ public void testFnCallSequence() { @Test @Category(RunnableOnService.class) public void testFnCallSequenceMulti() { - TestPipeline p = TestPipeline.create(); PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4))) .and(p.apply("Polite", Create.of(3, 5, 6, 7))) .apply(Flatten.pCollections()) @@ -206,7 +207,6 @@ public void after() { @Test @Category(NeedsRunner.class) public void testTeardownCalledAfterExceptionInSetup() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP); p .apply(Create.of(1, 2, 3)) @@ -227,7 +227,6 @@ public void testTeardownCalledAfterExceptionInSetup() { @Test @Category(NeedsRunner.class) public void testTeardownCalledAfterExceptionInStartBundle() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE); p .apply(Create.of(1, 2, 3)) @@ -246,7 +245,6 @@ public void testTeardownCalledAfterExceptionInStartBundle() { @Test @Category(NeedsRunner.class) public void testTeardownCalledAfterExceptionInProcessElement() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT); p .apply(Create.of(1, 2, 3)) @@ -265,7 +263,6 @@ public void testTeardownCalledAfterExceptionInProcessElement() { @Test @Category(NeedsRunner.class) public void testTeardownCalledAfterExceptionInFinishBundle() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE); p .apply(Create.of(1, 2, 3)) @@ -284,7 +281,6 @@ public void testTeardownCalledAfterExceptionInFinishBundle() { @Test @Category(NeedsRunner.class) public void testWithContextTeardownCalledAfterExceptionInSetup() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP); p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); try { @@ -300,7 +296,6 @@ public void testWithContextTeardownCalledAfterExceptionInSetup() { @Test @Category(NeedsRunner.class) public void testWithContextTeardownCalledAfterExceptionInStartBundle() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE); p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); try { @@ -316,7 +311,6 @@ public void testWithContextTeardownCalledAfterExceptionInStartBundle() { @Test @Category(NeedsRunner.class) public void testWithContextTeardownCalledAfterExceptionInProcessElement() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT); p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); try { @@ -332,7 +326,6 @@ public void testWithContextTeardownCalledAfterExceptionInProcessElement() { @Test @Category(NeedsRunner.class) public void testWithContextTeardownCalledAfterExceptionInFinishBundle() { - TestPipeline p = TestPipeline.create(); ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE); p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn)); try { 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 4a3e2ddb6a9d..3a47fc7149c4 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 @@ -47,7 +47,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.VarIntCoder; @@ -106,6 +105,9 @@ public class ParDoTest implements Serializable { // This test is Serializable, just so that it's easy to have // anonymous inner classes inside the non-static test methods. + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -302,7 +304,6 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(RunnableOnService.class) public void testParDo() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -319,7 +320,6 @@ public void testParDo() { @Test @Category(RunnableOnService.class) public void testParDo2() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -336,7 +336,6 @@ public void testParDo2() { @Test @Category(RunnableOnService.class) public void testParDoEmpty() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(); @@ -354,8 +353,6 @@ public void testParDoEmpty() { @Category(RunnableOnService.class) public void testParDoEmptyOutputs() { - Pipeline pipeline = TestPipeline.create(); - List inputs = Arrays.asList(); PCollection output = pipeline @@ -370,7 +367,6 @@ public void testParDoEmptyOutputs() { @Test @Category(RunnableOnService.class) public void testParDoWithSideOutputs() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -413,7 +409,6 @@ public void testParDoWithSideOutputs() { @Test @Category(RunnableOnService.class) public void testParDoEmptyWithSideOutputs() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(); @@ -454,7 +449,6 @@ public void testParDoEmptyWithSideOutputs() { @Test @Category(RunnableOnService.class) public void testParDoWithEmptySideOutputs() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(); @@ -482,7 +476,6 @@ public void testParDoWithEmptySideOutputs() { @Test @Category(RunnableOnService.class) public void testParDoWithOnlySideOutputs() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -507,7 +500,6 @@ public void processElement(ProcessContext c) { @Test @Category(NeedsRunner.class) public void testParDoWritingToUndeclaredSideOutput() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -529,7 +521,7 @@ public void testParDoWritingToUndeclaredSideOutput() { // TODO: The exception thrown is runner-specific, even if the behavior is general @Category(NeedsRunner.class) public void testParDoUndeclaredSideOutputLimit() { - Pipeline pipeline = TestPipeline.create(); + PCollection input = pipeline.apply(Create.of(Arrays.asList(3))); // Success for a total of 1000 outputs. @@ -566,7 +558,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testParDoWithSideInputs() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -598,7 +589,6 @@ public void testParDoWithSideInputs() { @Test @Category(RunnableOnService.class) public void testParDoWithSideInputsIsCumulative() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -632,7 +622,6 @@ public void testParDoWithSideInputsIsCumulative() { @Test @Category(RunnableOnService.class) public void testMultiOutputParDoWithSideInputs() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -670,7 +659,6 @@ public void testMultiOutputParDoWithSideInputs() { @Test @Category(RunnableOnService.class) public void testMultiOutputParDoWithSideInputsIsCumulative() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -708,7 +696,6 @@ public void testMultiOutputParDoWithSideInputsIsCumulative() { @Test @Category(NeedsRunner.class) public void testParDoReadingFromUnknownSideInput() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -746,7 +733,6 @@ public void testSideInputsWithMultipleWindows() { // on an input where the element is in multiple windows. The complication is // that side inputs are per-window, so the runner has to make sure // to process each window individually. - Pipeline p = TestPipeline.create(); MutableDateTime mutableNow = Instant.now().toMutableDateTime(); mutableNow.setMillisOfSecond(0); @@ -754,9 +740,9 @@ public void testSideInputsWithMultipleWindows() { SlidingWindows windowFn = SlidingWindows.of(Duration.standardSeconds(5)).every(Duration.standardSeconds(1)); - PCollectionView view = p.apply(Create.of(1)).apply(View.asSingleton()); + PCollectionView view = pipeline.apply(Create.of(1)).apply(View.asSingleton()); PCollection res = - p.apply(Create.timestamped(TimestampedValue.of("a", now))) + pipeline.apply(Create.timestamped(TimestampedValue.of("a", now))) .apply(Window.into(windowFn)) .apply(ParDo.of(new FnWithSideInputs(view)).withSideInputs(view)); @@ -766,14 +752,12 @@ public void testSideInputsWithMultipleWindows() { PAssert.that(res).inWindow(window).containsInAnyOrder("a:1"); } - p.run(); + pipeline.run(); } @Test @Category(NeedsRunner.class) public void testParDoWithErrorInStartBatch() { - Pipeline pipeline = TestPipeline.create(); - List inputs = Arrays.asList(3, -42, 666); pipeline.apply(Create.of(inputs)) @@ -787,7 +771,6 @@ public void testParDoWithErrorInStartBatch() { @Test @Category(NeedsRunner.class) public void testParDoWithErrorInProcessElement() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -802,7 +785,6 @@ public void testParDoWithErrorInProcessElement() { @Test @Category(NeedsRunner.class) public void testParDoWithErrorInFinishBatch() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -816,23 +798,27 @@ public void testParDoWithErrorInFinishBatch() { @Test public void testParDoOutputNameBasedOnDoFnWithTrimmedSuffix() { - Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of(1)).apply(ParDo.of(new TestDoFn())); + pipeline.enableAbandonedNodeEnforcement(false); + + PCollection output = pipeline.apply(Create.of(1)).apply(ParDo.of(new TestDoFn())); assertThat(output.getName(), containsString("ParDo(Test)")); } @Test public void testParDoOutputNameBasedOnLabel() { - Pipeline p = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + PCollection output = - p.apply(Create.of(1)).apply("MyParDo", ParDo.of(new TestDoFn())); + pipeline.apply(Create.of(1)).apply("MyParDo", ParDo.of(new TestDoFn())); assertThat(output.getName(), containsString("MyParDo")); } @Test public void testParDoOutputNameBasedDoFnWithoutMatchingSuffix() { - Pipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of(1)).apply(ParDo.of(new StrangelyNamedDoer())); + pipeline.enableAbandonedNodeEnforcement(false); + + PCollection output = + pipeline.apply(Create.of(1)).apply(ParDo.of(new StrangelyNamedDoer())); assertThat(output.getName(), containsString("ParDo(StrangelyNamedDoer)")); } @@ -850,7 +836,7 @@ public void testParDoMultiNameBasedDoFnWithTrimmerSuffix() { @Test public void testParDoWithSideOutputsName() { - Pipeline p = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); TupleTag mainOutputTag = new TupleTag("main"){}; TupleTag sideOutputTag1 = new TupleTag("side1"){}; @@ -858,7 +844,7 @@ public void testParDoWithSideOutputsName() { TupleTag sideOutputTag3 = new TupleTag("side3"){}; TupleTag sideOutputTagUnwritten = new TupleTag("sideUnwritten"){}; - PCollectionTuple outputs = p + PCollectionTuple outputs = pipeline .apply(Create.of(Arrays.asList(3, -42, 666))).setName("MyInput") .apply("MyParDo", ParDo .of(new TestDoFn( @@ -880,7 +866,6 @@ public void testParDoWithSideOutputsName() { @Test @Category(RunnableOnService.class) public void testParDoInCustomTransform() { - Pipeline pipeline = TestPipeline.create(); List inputs = Arrays.asList(3, -42, 666); @@ -904,7 +889,6 @@ public PCollection expand(PCollection input) { @Test @Category(NeedsRunner.class) public void testMultiOutputChaining() { - Pipeline pipeline = TestPipeline.create(); PCollectionTuple filters = pipeline .apply(Create.of(Arrays.asList(3, 4, 5, 6))) @@ -1106,7 +1090,7 @@ public Void apply(Iterable outputs) { @Test @Category(NeedsRunner.class) public void testSideOutputUnknownCoder() throws Exception { - Pipeline pipeline = TestPipeline.create(); + PCollection input = pipeline .apply(Create.of(Arrays.asList(1, 2, 3))); @@ -1122,7 +1106,8 @@ public void testSideOutputUnknownCoder() throws Exception { @Test public void testSideOutputUnregisteredExplicitCoder() throws Exception { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); + PCollection input = pipeline .apply(Create.of(Arrays.asList(1, 2, 3))); @@ -1144,7 +1129,7 @@ public void testSideOutputUnregisteredExplicitCoder() throws Exception { @Test @Category(NeedsRunner.class) public void testMainOutputUnregisteredExplicitCoder() { - Pipeline pipeline = TestPipeline.create(); + PCollection input = pipeline .apply(Create.of(Arrays.asList(1, 2, 3))); @@ -1165,7 +1150,6 @@ public void testMainOutputApplySideOutputNoCoder() { // should not cause a crash based on lack of a coder for the // side output. - Pipeline pipeline = TestPipeline.create(); final TupleTag mainOutputTag = new TupleTag("main"); final TupleTag sideOutputTag = new TupleTag("side"); PCollectionTuple tuple = pipeline @@ -1204,7 +1188,6 @@ public void processElement(ProcessContext context) { @Test @Category(NeedsRunner.class) public void testParDoOutputWithTimestamp() { - Pipeline pipeline = TestPipeline.create(); PCollection input = pipeline.apply(Create.of(Arrays.asList(3, 42, 6))); @@ -1226,7 +1209,6 @@ public void testParDoOutputWithTimestamp() { @Test @Category(NeedsRunner.class) public void testParDoSideOutputWithTimestamp() { - Pipeline pipeline = TestPipeline.create(); PCollection input = pipeline.apply(Create.of(Arrays.asList(3, 42, 6))); @@ -1258,7 +1240,6 @@ public void processElement(ProcessContext c) { @Test @Category(NeedsRunner.class) public void testParDoShiftTimestamp() { - Pipeline pipeline = TestPipeline.create(); PCollection input = pipeline.apply(Create.of(Arrays.asList(3, 42, 6))); @@ -1281,7 +1262,6 @@ public void testParDoShiftTimestamp() { @Test @Category(NeedsRunner.class) public void testParDoShiftTimestampInvalid() { - Pipeline pipeline = TestPipeline.create(); pipeline.apply(Create.of(Arrays.asList(3, 42, 6))) .apply(ParDo.of(new TestOutputTimestampDoFn())) @@ -1300,7 +1280,6 @@ public void testParDoShiftTimestampInvalid() { @Test @Category(NeedsRunner.class) public void testParDoShiftTimestampInvalidZeroAllowed() { - Pipeline pipeline = TestPipeline.create(); pipeline.apply(Create.of(Arrays.asList(3, 42, 6))) .apply(ParDo.of(new TestOutputTimestampDoFn())) @@ -1353,7 +1332,6 @@ public Void apply(Iterable input) { @Test @Category(RunnableOnService.class) public void testWindowingInStartAndFinishBundle() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline @@ -1391,7 +1369,6 @@ public void finishBundle(Context c) { @Test @Category(NeedsRunner.class) public void testWindowingInStartBundleException() { - Pipeline pipeline = TestPipeline.create(); pipeline .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) @@ -1477,13 +1454,12 @@ public void processElement( } }; - Pipeline p = TestPipeline.create(); PCollection output = - p.apply(Create.of(KV.of("hello", 42), KV.of("hello", 97), KV.of("hello", 84))) + pipeline.apply(Create.of(KV.of("hello", 42), KV.of("hello", 97), KV.of("hello", 84))) .apply(ParDo.of(fn)); PAssert.that(output).containsInAnyOrder(0, 1, 2); - p.run(); + pipeline.run(); } @Test @@ -1514,9 +1490,8 @@ public void processElement( } }; - Pipeline p = TestPipeline.create(); PCollectionTuple output = - p.apply( + pipeline.apply( Create.of( KV.of("hello", 42), KV.of("hello", 97), @@ -1534,7 +1509,7 @@ public void processElement( // There are 1 and 3 from "hello" and just "1" from "goodbye" PAssert.that(odds).containsInAnyOrder(1, 3, 1); - p.run(); + pipeline.run(); } @Test @@ -1562,24 +1537,23 @@ public void processElement( } }; - Pipeline p = TestPipeline.create(); PCollection> output = - p.apply( + pipeline.apply( Create.of( KV.of("hello", 97), KV.of("hello", 42), KV.of("hello", 84), KV.of("hello", 12))) .apply(ParDo.of(fn)); PAssert.that(output).containsInAnyOrder(Lists.newArrayList(12, 42, 84, 97)); - p.run(); + pipeline.run(); } @Test @Category({RunnableOnService.class, UsesStatefulParDo.class}) public void testBagStateSideInput() { - Pipeline p = TestPipeline.create(); final PCollectionView> listView = - p.apply("Create list for side input", Create.of(2, 1, 0)).apply(View.asList()); + pipeline + .apply("Create list for side input", Create.of(2, 1, 0)).apply(View.asList()); final String stateId = "foo"; DoFn, List> fn = @@ -1607,7 +1581,7 @@ public void processElement( }; PCollection> output = - p.apply( + pipeline.apply( "Create main input", Create.of( KV.of("hello", 97), KV.of("hello", 42), KV.of("hello", 84), KV.of("hello", 12))) @@ -1616,7 +1590,7 @@ public void processElement( PAssert.that(output).containsInAnyOrder( Lists.newArrayList(12, 42, 84, 97), Lists.newArrayList(0, 1, 2)); - p.run(); + pipeline.run(); } /** @@ -1658,11 +1632,9 @@ public void onTimer(OnTimerContext context) { } }; - Pipeline p = TestPipeline.create(); - - PCollection output = p.apply(Create.of(KV.of("hello", 37))).apply(ParDo.of(fn)); + PCollection output = pipeline.apply(Create.of(KV.of("hello", 37))).apply(ParDo.of(fn)); PAssert.that(output).containsInAnyOrder(3, 42); - p.run(); + pipeline.run(); } @Test @@ -1704,7 +1676,6 @@ public SomeTracker newTracker(Object restriction) { @Test public void testRejectsWrongWindowType() { - Pipeline p = TestPipeline.create(); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(GlobalWindow.class.getSimpleName()); @@ -1712,7 +1683,8 @@ public void testRejectsWrongWindowType() { thrown.expectMessage("window type"); thrown.expectMessage("not a supertype"); - p.apply(Create.of(1, 2, 3)) + pipeline + .apply(Create.of(1, 2, 3)) .apply( ParDo.of( new DoFn() { @@ -1735,9 +1707,8 @@ public void process(ProcessContext c, IntervalWindow w) {} public void testMultipleWindowSubtypesOK() { final String timerId = "gobbledegook"; - Pipeline p = TestPipeline.create(); - - p.apply(Create.of(1, 2, 3)) + pipeline + .apply(Create.of(1, 2, 3)) .apply(Window.into(FixedWindows.of(Duration.standardSeconds(10)))) .apply( ParDo.of( @@ -1759,26 +1730,25 @@ public void onTimer(BoundedWindow w) {} public void testRejectsSplittableDoFnByDefault() { // ParDo with a splittable DoFn must be overridden by the runner. // Without an override, applying it directly must fail. - Pipeline p = TestPipeline.create(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(p.getRunner().getClass().getName()); + thrown.expectMessage(pipeline.getRunner().getClass().getName()); thrown.expectMessage("does not support Splittable DoFn"); - p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new TestSplittableDoFn())); + pipeline.apply(Create.of(1, 2, 3)).apply(ParDo.of(new TestSplittableDoFn())); } @Test public void testMultiRejectsSplittableDoFnByDefault() { // ParDo with a splittable DoFn must be overridden by the runner. // Without an override, applying it directly must fail. - Pipeline p = TestPipeline.create(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(p.getRunner().getClass().getName()); + thrown.expectMessage(pipeline.getRunner().getClass().getName()); thrown.expectMessage("does not support Splittable DoFn"); - p.apply(Create.of(1, 2, 3)) + pipeline + .apply(Create.of(1, 2, 3)) .apply( ParDo.of(new TestSplittableDoFn()) .withOutputTags( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java index 1cbe344adc54..87d7460fe2c4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java @@ -25,7 +25,6 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -47,6 +46,7 @@ @RunWith(JUnit4.class) public class PartitionTest implements Serializable { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); @Rule public transient ExpectedException thrown = ExpectedException.none(); static class ModFn implements PartitionFn { @@ -63,10 +63,10 @@ public int partitionFor(Integer elem, int numPartitions) { } } + @Test @Category(RunnableOnService.class) public void testEvenOddPartition() { - Pipeline pipeline = TestPipeline.create(); PCollectionList outputs = pipeline .apply(Create.of(591, 11789, 1257, 24578, 24799, 307)) @@ -81,7 +81,6 @@ public void testEvenOddPartition() { @Test @Category(NeedsRunner.class) public void testModPartition() { - Pipeline pipeline = TestPipeline.create(); PCollectionList outputs = pipeline .apply(Create.of(1, 2, 4, 5)) @@ -96,7 +95,6 @@ public void testModPartition() { @Test @Category(NeedsRunner.class) public void testOutOfBoundsPartitions() { - Pipeline pipeline = TestPipeline.create(); pipeline .apply(Create.of(-1)) @@ -110,7 +108,6 @@ public void testOutOfBoundsPartitions() { @Test public void testZeroNumPartitions() { - Pipeline pipeline = TestPipeline.create(); PCollection input = pipeline.apply(Create.of(591)); @@ -122,7 +119,6 @@ public void testZeroNumPartitions() { @Test @Category(NeedsRunner.class) public void testDroppedPartition() { - Pipeline pipeline = TestPipeline.create(); // Compute the set of integers either 1 or 2 mod 3, the hard way. PCollectionList outputs = pipeline diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java index 6e196b46d554..cd707daa4a39 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -33,11 +34,13 @@ /** Tests for {@link Regex}. */ @RunWith(JUnit4.class) public class RegexTest implements Serializable { + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test @Category(NeedsRunner.class) public void testFind() { - TestPipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of("aj", "xj", "yj", "zj")).apply(Regex.find("[xyz]")); @@ -48,8 +51,6 @@ public void testFind() { @Test @Category(NeedsRunner.class) public void testFindGroup() { - TestPipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of("aj", "xj", "yj", "zj")).apply(Regex.find("([xyz])", 1)); @@ -60,8 +61,6 @@ public void testFindGroup() { @Test @Category(NeedsRunner.class) public void testFindNone() { - TestPipeline p = TestPipeline.create(); - PCollection output = p.apply(Create.of("a", "b", "c", "d")).apply(Regex.find("[xyz]")); PAssert.that(output).empty(); @@ -71,7 +70,6 @@ public void testFindNone() { @Test @Category(NeedsRunner.class) public void testKVFind() { - TestPipeline p = TestPipeline.create(); PCollection> output = p.apply(Create.of("a b c")).apply(Regex.findKV("a (b) (c)", 1, 2)); @@ -83,7 +81,6 @@ public void testKVFind() { @Test @Category(NeedsRunner.class) public void testKVFindNone() { - TestPipeline p = TestPipeline.create(); PCollection> output = p.apply(Create.of("x y z")).apply(Regex.findKV("a (b) (c)", 1, 2)); @@ -95,7 +92,6 @@ public void testKVFindNone() { @Test @Category(NeedsRunner.class) public void testMatches() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("a", "x", "y", "z")).apply(Regex.matches("[xyz]")); @@ -107,7 +103,6 @@ public void testMatches() { @Test @Category(NeedsRunner.class) public void testMatchesNone() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("a", "b", "c", "d")).apply(Regex.matches("[xyz]")); @@ -119,7 +114,6 @@ public void testMatchesNone() { @Test @Category(NeedsRunner.class) public void testMatchesGroup() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("a", "x xxx", "x yyy", "x zzz")).apply(Regex.matches("x ([xyz]*)", 1)); @@ -131,7 +125,6 @@ public void testMatchesGroup() { @Test @Category(NeedsRunner.class) public void testKVMatches() { - TestPipeline p = TestPipeline.create(); PCollection> output = p.apply(Create.of("a b c")).apply(Regex.matchesKV("a (b) (c)", 1, 2)); @@ -143,7 +136,6 @@ public void testKVMatches() { @Test @Category(NeedsRunner.class) public void testKVMatchesNone() { - TestPipeline p = TestPipeline.create(); PCollection> output = p.apply(Create.of("x y z")).apply(Regex.matchesKV("a (b) (c)", 1, 2)); @@ -154,7 +146,6 @@ public void testKVMatchesNone() { @Test @Category(NeedsRunner.class) public void testReplaceAll() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("xj", "yj", "zj")).apply(Regex.replaceAll("[xyz]", "new")); @@ -166,7 +157,6 @@ public void testReplaceAll() { @Test @Category(NeedsRunner.class) public void testReplaceAllMixed() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("abc", "xj", "yj", "zj", "def")).apply(Regex.replaceAll("[xyz]", "new")); @@ -178,7 +168,6 @@ public void testReplaceAllMixed() { @Test @Category(NeedsRunner.class) public void testReplaceFirst() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("xjx", "yjy", "zjz")).apply(Regex.replaceFirst("[xyz]", "new")); @@ -190,7 +179,6 @@ public void testReplaceFirst() { @Test @Category(NeedsRunner.class) public void testReplaceFirstMixed() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("abc", "xjx", "yjy", "zjz", "def")) @@ -203,7 +191,6 @@ public void testReplaceFirstMixed() { @Test @Category(NeedsRunner.class) public void testSplits() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("The quick brown fox jumps over the lazy dog")) @@ -217,7 +204,6 @@ public void testSplits() { @Test @Category(NeedsRunner.class) public void testSplitsWithEmpty() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("The quick brown fox jumps over the lazy dog")) @@ -235,7 +221,6 @@ public void testSplitsWithEmpty() { @Test @Category(NeedsRunner.class) public void testSplitsWithoutEmpty() { - TestPipeline p = TestPipeline.create(); PCollection output = p.apply(Create.of("The quick brown fox jumps over the lazy dog")) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java index a0555fa60800..9cc12d4e2299 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.Set; import java.util.TreeSet; + import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -41,6 +42,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -102,71 +104,70 @@ public Void apply(Iterable in) { } } + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testSample() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(DATA) + PCollection input = pipeline.apply(Create.of(DATA) .withCoder(BigEndianIntegerCoder.of())); PCollection> output = input.apply( Sample.fixedSizeGlobally(3)); PAssert.thatSingletonIterable(output) .satisfies(new VerifyCorrectSample<>(3, DATA)); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testSampleEmpty() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(EMPTY) + PCollection input = pipeline.apply(Create.of(EMPTY) .withCoder(BigEndianIntegerCoder.of())); PCollection> output = input.apply( Sample.fixedSizeGlobally(3)); PAssert.thatSingletonIterable(output) .satisfies(new VerifyCorrectSample<>(0, EMPTY)); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testSampleZero() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(DATA) + PCollection input = pipeline.apply(Create.of(DATA) .withCoder(BigEndianIntegerCoder.of())); PCollection> output = input.apply( Sample.fixedSizeGlobally(0)); PAssert.thatSingletonIterable(output) .satisfies(new VerifyCorrectSample<>(0, DATA)); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testSampleInsufficientElements() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(DATA) + PCollection input = pipeline.apply(Create.of(DATA) .withCoder(BigEndianIntegerCoder.of())); PCollection> output = input.apply( Sample.fixedSizeGlobally(10)); PAssert.thatSingletonIterable(output) .satisfies(new VerifyCorrectSample<>(5, DATA)); - p.run(); + pipeline.run(); } @Test(expected = IllegalArgumentException.class) public void testSampleNegative() { - Pipeline p = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(false); - PCollection input = p.apply(Create.of(DATA) + PCollection input = pipeline.apply(Create.of(DATA) .withCoder(BigEndianIntegerCoder.of())); input.apply(Sample.fixedSizeGlobally(-1)); } @@ -174,9 +175,8 @@ public void testSampleNegative() { @Test @Category(RunnableOnService.class) public void testSampleMultiplicity() { - Pipeline p = TestPipeline.create(); - PCollection input = p.apply(Create.of(REPEATED_DATA) + PCollection input = pipeline.apply(Create.of(REPEATED_DATA) .withCoder(BigEndianIntegerCoder.of())); // At least one value must be selected with multiplicity. PCollection> output = input.apply( @@ -184,7 +184,7 @@ public void testSampleMultiplicity() { PAssert.thatSingletonIterable(output) .satisfies(new VerifyCorrectSample<>(6, REPEATED_DATA)); - p.run(); + pipeline.run(); } private static class VerifyAnySample implements SerializableFunction, Void> { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java index 022c2e5cd4b4..e3b58b7b9bec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -54,6 +53,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.MutableDateTime; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -151,10 +151,13 @@ public void process(ProcessContext c) { } } + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testPairWithIndexBasic() { - Pipeline p = TestPipeline.create(); + PCollection> res = p.apply(Create.of("a", "bb", "ccccc")) .apply(ParDo.of(new PairStringWithIndexToLength())) @@ -180,7 +183,6 @@ public void testPairWithIndexBasic() { public void testPairWithIndexWindowedTimestamped() { // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps // of elements in the input collection. - Pipeline p = TestPipeline.create(); MutableDateTime mutableNow = Instant.now().toMutableDateTime(); mutableNow.setMillisOfSecond(0); @@ -277,7 +279,6 @@ public OffsetRangeTracker newTracker(OffsetRange range) { @Test @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testOutputAfterCheckpoint() throws Exception { - Pipeline p = TestPipeline.create(); PCollection outputs = p.apply(Create.of("foo")) .apply(ParDo.of(new SDFWithMultipleOutputsPerBlock())); PAssert.thatSingleton(outputs.apply(Count.globally())) @@ -317,7 +318,6 @@ public OffsetRangeTracker newTracker(OffsetRange range) { @Test @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testSideInputsAndOutputs() throws Exception { - Pipeline p = TestPipeline.create(); PCollectionView sideInput = p.apply("side input", Create.of("foo")).apply(View.asSingleton()); @@ -344,7 +344,6 @@ public void testSideInputsAndOutputs() throws Exception { @Test @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testLateData() throws Exception { - Pipeline p = TestPipeline.create(); Instant base = Instant.now(); @@ -439,7 +438,6 @@ public void tearDown() { @Test @Category({RunnableOnService.class, UsesSplittableParDo.class}) public void testLifecycleMethods() throws Exception { - Pipeline p = TestPipeline.create(); PCollection res = p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java index b6242524b6a3..d0111970befb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java @@ -52,6 +52,9 @@ @RunWith(JUnit4.class) public class TopTest { + @Rule + public final TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException expectedEx = ExpectedException.none(); @@ -93,7 +96,6 @@ public PCollection> createEmptyInputTable(Pipeline p) { @Category(NeedsRunner.class) @SuppressWarnings("unchecked") public void testTop() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(COLLECTION)) .withCoder(StringUtf8Coder.of())); @@ -125,7 +127,6 @@ public void testTop() { @Category(NeedsRunner.class) @SuppressWarnings("unchecked") public void testTopEmpty() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(EMPTY_COLLECTION)) .withCoder(StringUtf8Coder.of())); @@ -151,7 +152,8 @@ public void testTopEmpty() { @Test public void testTopEmptyWithIncompatibleWindows() { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); + Bound windowingFn = Window.into(FixedWindows.of(Duration.standardDays(10L))); PCollection input = p.apply(Create.timestamped(Collections.emptyList(), Collections.emptyList())) @@ -170,7 +172,6 @@ public void testTopEmptyWithIncompatibleWindows() { @Category(NeedsRunner.class) @SuppressWarnings("unchecked") public void testTopZero() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(COLLECTION)) .withCoder(StringUtf8Coder.of())); @@ -202,7 +203,8 @@ public void testTopZero() { // This is a purely compile-time test. If the code compiles, then it worked. @Test public void testPerKeySerializabilityRequirement() { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); + p.apply("CreateCollection", Create.of(Arrays.asList(COLLECTION)) .withCoder(StringUtf8Coder.of())); @@ -218,7 +220,8 @@ public void testPerKeySerializabilityRequirement() { @Test public void testCountConstraint() { - Pipeline p = TestPipeline.create(); + p.enableAbandonedNodeEnforcement(false); + PCollection input = p.apply(Create.of(Arrays.asList(COLLECTION)) .withCoder(StringUtf8Coder.of())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java index 0bf2e2e9232e..5e27552dfad5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ValuesTest.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals; import java.util.Arrays; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -29,6 +28,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -52,10 +52,12 @@ public class ValuesTest { static final KV[] EMPTY_TABLE = new KV[] { }; + @Rule + public final TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testValues() { - Pipeline p = TestPipeline.create(); PCollection> input = p.apply(Create.of(Arrays.asList(TABLE)).withCoder( @@ -72,7 +74,6 @@ public void testValues() { @Test @Category(RunnableOnService.class) public void testValuesEmpty() { - Pipeline p = TestPipeline.create(); PCollection> input = p.apply(Create.of(Arrays.asList(EMPTY_TABLE)).withCoder( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index 3bf63fdbec63..1d8b32c92270 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -83,13 +83,15 @@ public class ViewTest implements Serializable { // This test is Serializable, just so that it's easy to have // anonymous inner classes inside the non-static test methods. + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test @Category(RunnableOnService.class) public void testSingletonSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView view = pipeline.apply("Create47", Create.of(47)).apply(View.asSingleton()); @@ -112,7 +114,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedSingletonSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView view = pipeline.apply("Create47", Create.timestamped( @@ -143,7 +144,6 @@ public void processElement(ProcessContext c) { @Test @Category(NeedsRunner.class) public void testEmptySingletonSideInput() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView view = pipeline.apply("CreateEmptyIntegers", Create.of().withCoder(VarIntCoder.of())) @@ -169,7 +169,6 @@ public void processElement(ProcessContext c) { @Test @Category(NeedsRunner.class) public void testNonSingletonSideInput() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection oneTwoThree = pipeline.apply(Create.of(1, 2, 3)); final PCollectionView view = oneTwoThree.apply(View.asSingleton()); @@ -194,7 +193,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testListSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(11, 13, 17, 23)).apply(View.asList()); @@ -221,7 +219,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedListSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.timestamped( @@ -262,7 +259,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testEmptyListSideInput() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateEmptyView", Create.of().withCoder(VarIntCoder.of())) @@ -289,7 +285,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testListSideInputIsImmutable() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(11)).apply(View.asList()); @@ -335,7 +330,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testIterableSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(11, 13, 17, 23)) @@ -361,7 +355,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedIterableSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.timestamped( @@ -401,7 +394,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testEmptyIterableSideInput() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateEmptyView", Create.of().withCoder(VarIntCoder.of())) @@ -427,7 +419,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testIterableSideInputIsImmutable() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(11)).apply(View.asIterable()); @@ -459,7 +450,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMultimapSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3))) @@ -487,7 +477,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMultimapAsEntrySetSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3))) @@ -539,7 +528,6 @@ public void verifyDeterministic() @Test @Category(RunnableOnService.class) public void testMultimapSideInputWithNonDeterministicKeyCoder() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", @@ -569,7 +557,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedMultimapSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", Create.timestamped( @@ -608,7 +595,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedMultimapAsEntrySetSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", Create.timestamped( @@ -651,7 +637,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedMultimapSideInputWithNonDeterministicKeyCoder() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", @@ -691,7 +676,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testEmptyMultimapSideInput() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateEmptyView", Create.>of().withCoder( @@ -720,7 +704,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testEmptyMultimapSideInputWithNonDeterministicKeyCoder() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateEmptyView", @@ -750,7 +733,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMultimapSideInputIsImmutable() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView>> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1))) @@ -798,7 +780,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMapSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("b", 3))) @@ -825,7 +806,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMapAsEntrySetSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("b", 3))) @@ -855,7 +835,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMapSideInputWithNonDeterministicKeyCoder() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", @@ -884,7 +863,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedMapSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.timestamped( @@ -922,7 +900,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedMapAsEntrySetSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.timestamped( @@ -964,7 +941,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedMapSideInputWithNonDeterministicKeyCoder() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", @@ -1004,7 +980,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testEmptyMapSideInput() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateEmptyView", Create.>of().withCoder( @@ -1033,7 +1008,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testEmptyMapSideInputWithNonDeterministicKeyCoder() throws Exception { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateEmptyView", Create.>of().withCoder( @@ -1062,7 +1036,6 @@ public void processElement(ProcessContext c) { @Test @Category(NeedsRunner.class) public void testMapSideInputWithNullValuesCatchesDuplicates() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline @@ -1098,7 +1071,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testMapSideInputIsImmutable() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1))) @@ -1145,7 +1117,6 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testCombinedMapSideInput() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view = pipeline.apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 20), KV.of("b", 3))) @@ -1172,10 +1143,9 @@ public void processElement(ProcessContext c) { @Test @Category(RunnableOnService.class) public void testWindowedSideInputFixedToFixed() { - Pipeline p = TestPipeline.create(); final PCollectionView view = - p.apply( + pipeline.apply( "CreateSideInput", Create.timestamped(TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(11)), TimestampedValue.of(3, new Instant(13)))) @@ -1184,7 +1154,7 @@ public void testWindowedSideInputFixedToFixed() { .apply(View.asSingleton()); PCollection output = - p.apply("CreateMainInput", Create.timestamped( + pipeline.apply("CreateMainInput", Create.timestamped( TimestampedValue.of("A", new Instant(4)), TimestampedValue.of("B", new Instant(15)), TimestampedValue.of("C", new Instant(7)))) @@ -1199,16 +1169,15 @@ public void processElement(ProcessContext c) { PAssert.that(output).containsInAnyOrder("A1", "B5", "C1"); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testWindowedSideInputFixedToGlobal() { - Pipeline p = TestPipeline.create(); final PCollectionView view = - p.apply( + pipeline.apply( "CreateSideInput", Create.timestamped(TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(11)), TimestampedValue.of(3, new Instant(13)))) @@ -1217,7 +1186,7 @@ public void testWindowedSideInputFixedToGlobal() { .apply(View.asSingleton()); PCollection output = - p.apply("CreateMainInput", Create.timestamped( + pipeline.apply("CreateMainInput", Create.timestamped( TimestampedValue.of("A", new Instant(4)), TimestampedValue.of("B", new Instant(15)), TimestampedValue.of("C", new Instant(7)))) @@ -1232,23 +1201,22 @@ public void processElement(ProcessContext c) { PAssert.that(output).containsInAnyOrder("A6", "B6", "C6"); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testWindowedSideInputFixedToFixedWithDefault() { - Pipeline p = TestPipeline.create(); final PCollectionView view = - p.apply("CreateSideInput", Create.timestamped( + pipeline.apply("CreateSideInput", Create.timestamped( TimestampedValue.of(2, new Instant(11)), TimestampedValue.of(3, new Instant(13)))) .apply("WindowSideInput", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersGlobally().asSingletonView()); PCollection output = - p.apply("CreateMainInput", Create.timestamped( + pipeline.apply("CreateMainInput", Create.timestamped( TimestampedValue.of("A", new Instant(4)), TimestampedValue.of("B", new Instant(15)), TimestampedValue.of("C", new Instant(7)))) @@ -1263,16 +1231,15 @@ public void processElement(ProcessContext c) { PAssert.that(output).containsInAnyOrder("A0", "B5", "C0"); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testSideInputWithNullDefault() { - Pipeline p = TestPipeline.create(); final PCollectionView view = - p.apply("CreateSideInput", Create.of((Void) null).withCoder(VoidCoder.of())) + pipeline.apply("CreateSideInput", Create.of((Void) null).withCoder(VoidCoder.of())) .apply(Combine.globally(new SerializableFunction, Void>() { @Override public Void apply(Iterable input) { @@ -1281,7 +1248,7 @@ public Void apply(Iterable input) { }).asSingletonView()); PCollection output = - p.apply("CreateMainInput", Create.of("")) + pipeline.apply("CreateMainInput", Create.of("")) .apply( "OutputMainAndSideInputs", ParDo.withSideInputs(view).of(new DoFn() { @@ -1293,13 +1260,12 @@ public void processElement(ProcessContext c) { PAssert.that(output).containsInAnyOrder("null"); - p.run(); + pipeline.run(); } @Test @Category(RunnableOnService.class) public void testSideInputWithNestedIterables() { - Pipeline pipeline = TestPipeline.create(); final PCollectionView> view1 = pipeline.apply("CreateVoid1", Create.of((Void) null).withCoder(VoidCoder.of())) .apply("OutputOneInteger", ParDo.of(new DoFn() { @@ -1386,51 +1352,51 @@ private void testViewNonmerging( @Test public void testViewUnboundedAsSingletonDirect() { - testViewUnbounded(TestPipeline.create(), View.>asSingleton()); + testViewUnbounded(pipeline, View.>asSingleton()); } @Test public void testViewUnboundedAsIterableDirect() { - testViewUnbounded(TestPipeline.create(), View.>asIterable()); + testViewUnbounded(pipeline, View.>asIterable()); } @Test public void testViewUnboundedAsListDirect() { - testViewUnbounded(TestPipeline.create(), View.>asList()); + testViewUnbounded(pipeline, View.>asList()); } @Test public void testViewUnboundedAsMapDirect() { - testViewUnbounded(TestPipeline.create(), View.asMap()); + testViewUnbounded(pipeline, View.asMap()); } @Test public void testViewUnboundedAsMultimapDirect() { - testViewUnbounded(TestPipeline.create(), View.asMultimap()); + testViewUnbounded(pipeline, View.asMultimap()); } @Test public void testViewNonmergingAsSingletonDirect() { - testViewNonmerging(TestPipeline.create(), View.>asSingleton()); + testViewNonmerging(pipeline, View.>asSingleton()); } @Test public void testViewNonmergingAsIterableDirect() { - testViewNonmerging(TestPipeline.create(), View.>asIterable()); + testViewNonmerging(pipeline, View.>asIterable()); } @Test public void testViewNonmergingAsListDirect() { - testViewNonmerging(TestPipeline.create(), View.>asList()); + testViewNonmerging(pipeline, View.>asList()); } @Test public void testViewNonmergingAsMapDirect() { - testViewNonmerging(TestPipeline.create(), View.asMap()); + testViewNonmerging(pipeline, View.asMap()); } @Test public void testViewNonmergingAsMultimapDirect() { - testViewNonmerging(TestPipeline.create(), View.asMultimap()); + testViewNonmerging(pipeline, View.asMultimap()); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java index f958807fcf77..8abbf1a5b7fc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithKeysTest.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -29,6 +28,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -63,10 +63,12 @@ public class WithKeysTest { KV.of(100, "bbb") ); + @Rule + public final TestPipeline p = TestPipeline.create(); + @Test @Category(NeedsRunner.class) public void testExtractKeys() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder( @@ -83,7 +85,6 @@ public void testExtractKeys() { @Test @Category(NeedsRunner.class) public void testConstantKeys() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder( @@ -105,7 +106,6 @@ public void testWithKeysGetName() { @Test @Category(NeedsRunner.class) public void testWithKeysWithUnneededWithKeyTypeSucceeds() { - TestPipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java index 923b97c52ee3..67a265856015 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java @@ -41,13 +41,16 @@ */ @RunWith(JUnit4.class) public class WithTimestampsTest implements Serializable { + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test @Category(RunnableOnService.class) public void withTimestampsShouldApplyTimestamps() { - TestPipeline p = TestPipeline.create(); SerializableFunction timestampFn = new SerializableFunction() { @@ -86,7 +89,6 @@ public void processElement(DoFn>.ProcessContext c) @Test @Category(NeedsRunner.class) public void withTimestampsBackwardsInTimeShouldThrow() { - TestPipeline p = TestPipeline.create(); SerializableFunction timestampFn = new SerializableFunction() { @@ -120,7 +122,6 @@ public Instant apply(String input) { @Test @Category(RunnableOnService.class) public void withTimestampsBackwardsInTimeAndWithAllowedTimestampSkewShouldSucceed() { - TestPipeline p = TestPipeline.create(); SerializableFunction timestampFn = new SerializableFunction() { @@ -181,7 +182,6 @@ public Instant apply(String input) { } }; - TestPipeline p = TestPipeline.create(); String yearTwoThousand = "946684800000"; p.apply(Create.of("1234", "0", Integer.toString(Integer.MAX_VALUE), yearTwoThousand)) .apply(WithTimestamps.of(timestampFn)); @@ -197,7 +197,6 @@ public Instant apply(String input) { @Test @Category(RunnableOnService.class) public void withTimestampsWithNullFnShouldThrowOnConstruction() { - TestPipeline p = TestPipeline.create(); SerializableFunction timestampFn = null; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java index e8c8b15ddbfd..0e5c177447be 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java @@ -47,6 +47,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -118,14 +119,15 @@ private PCollection> buildGetOnlyGbk( return coGbkResults; } + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testCoGroupByKeyGetOnly() { final TupleTag tag1 = new TupleTag<>(); final TupleTag tag2 = new TupleTag<>(); - Pipeline p = TestPipeline.create(); - PCollection> coGbkResults = buildGetOnlyGbk(p, tag1, tag2); @@ -264,7 +266,6 @@ public void testCoGroupByKey() { final TupleTag addressesTag = new TupleTag<>(); final TupleTag purchasesTag = new TupleTag<>(); - Pipeline p = TestPipeline.create(); PCollection> coGbkResults = buildPurchasesCoGbk(p, purchasesTag, addressesTag, namesTag); @@ -456,8 +457,6 @@ public void testCoGroupByKeyHandleResults() { TupleTag addressesTag = new TupleTag<>(); TupleTag purchasesTag = new TupleTag<>(); - Pipeline p = TestPipeline.create(); - PCollection> coGbkResults = buildPurchasesCoGbk(p, purchasesTag, addressesTag, namesTag); @@ -486,8 +485,6 @@ public void testCoGroupByKeyWithWindowing() { TupleTag clicksTag = new TupleTag<>(); TupleTag purchasesTag = new TupleTag<>(); - Pipeline p = TestPipeline.create(); - PCollection> coGbkResults = buildPurchasesCoGbkWithWindowing(p, clicksTag, purchasesTag); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index 3125ae8b04d9..e21668e5182d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -29,7 +29,6 @@ import static org.mockito.Mockito.when; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -61,12 +60,16 @@ @RunWith(JUnit4.class) public class WindowTest implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create() + .enableAbandonedNodeEnforcement(false); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test public void testWindowIntoSetWindowfn() { - WindowingStrategy strategy = TestPipeline.create() + WindowingStrategy strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply(Window.into(FixedWindows.of(Duration.standardMinutes(10)))) .getWindowingStrategy(); @@ -79,7 +82,7 @@ public void testWindowIntoSetWindowfn() { public void testWindowIntoTriggersAndAccumulating() { FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10)); Repeatedly trigger = Repeatedly.forever(AfterPane.elementCountAtLeast(5)); - WindowingStrategy strategy = TestPipeline.create() + WindowingStrategy strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply(Window.into(fixed10) .triggering(trigger) @@ -96,7 +99,7 @@ public void testWindowIntoTriggersAndAccumulating() { public void testWindowPropagatesEachPart() { FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10)); Repeatedly trigger = Repeatedly.forever(AfterPane.elementCountAtLeast(5)); - WindowingStrategy strategy = TestPipeline.create() + WindowingStrategy strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply("Mode", Window.accumulatingFiredPanes()) .apply("Lateness", Window.withAllowedLateness(Duration.standardDays(1))) @@ -112,9 +115,10 @@ public void testWindowPropagatesEachPart() { @Test public void testWindowIntoPropagatesLateness() { + FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10)); FixedWindows fixed25 = FixedWindows.of(Duration.standardMinutes(25)); - WindowingStrategy strategy = TestPipeline.create() + WindowingStrategy strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply("WindowInto10", Window.into(fixed10) .withAllowedLateness(Duration.standardDays(1)) @@ -157,7 +161,7 @@ public void testMissingMode() { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("requires that the accumulation mode"); - TestPipeline.create() + pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply("Window", Window.into(fixed10)) .apply("Lateness", Window.withAllowedLateness(Duration.standardDays(1))) @@ -171,7 +175,7 @@ public void testMissingLateness() { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("requires that the allowed lateness"); - TestPipeline.create() + pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply("Mode", Window.accumulatingFiredPanes()) .apply("Window", Window.into(fixed10)) @@ -185,7 +189,7 @@ public void testMissingLateness() { @Test @Category(RunnableOnService.class) public void testOutputTimeFnDefault() { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(true); pipeline .apply( @@ -219,7 +223,7 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(RunnableOnService.class) public void testOutputTimeFnEndOfWindow() { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(true); pipeline.apply( Create.timestamped( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java index d4fab17fb244..f7ae5d8d74b4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java @@ -21,7 +21,6 @@ import java.io.FileOutputStream; import java.io.PrintStream; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.testing.NeedsRunner; @@ -50,6 +49,10 @@ /** Unit tests for bucketing. */ @RunWith(JUnit4.class) public class WindowingTest implements Serializable { + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -88,7 +91,6 @@ private String output(String value, int count, int timestamp, int windowStart, i @Test @Category(RunnableOnService.class) public void testPartitioningWindowing() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply( Create.timestamped( @@ -114,7 +116,6 @@ public void testPartitioningWindowing() { @Test @Category(RunnableOnService.class) public void testNonPartitioningWindowing() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply( Create.timestamped( @@ -140,7 +141,6 @@ public void testNonPartitioningWindowing() { @Test @Category(RunnableOnService.class) public void testMergingWindowing() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply( Create.timestamped( @@ -162,7 +162,6 @@ public void testMergingWindowing() { @Test @Category(RunnableOnService.class) public void testWindowPreservation() { - Pipeline p = TestPipeline.create(); PCollection input1 = p.apply("Create12", Create.timestamped( TimestampedValue.of("a", new Instant(1)), @@ -190,7 +189,6 @@ public void testWindowPreservation() { @Test @Category(NeedsRunner.class) public void testEmptyInput() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.timestamped() .withCoder(StringUtf8Coder.of())); @@ -218,7 +216,6 @@ public void testTextIoInput() throws Exception { writer.println("d 11"); } - Pipeline p = TestPipeline.create(); PCollection output = p.begin() .apply("ReadLines", TextIO.Read.from(filename)) .apply(ParDo.of(new ExtractWordsWithTimestampsFn())) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java index d990ee066e27..d47cddcb0d2f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -36,6 +35,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -65,10 +65,12 @@ public class ReshuffleTest { KV.of("k1", (Iterable) ImmutableList.of(3)), KV.of("k2", (Iterable) ImmutableList.of(4))); + @Rule + public final TestPipeline pipeline = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void testJustReshuffle() { - Pipeline pipeline = TestPipeline.create(); PCollection> input = pipeline .apply(Create.of(ARBITRARY_KVS) @@ -89,7 +91,6 @@ public void testJustReshuffle() { @Test @Category(RunnableOnService.class) public void testReshuffleAfterSessionsAndGroupByKey() { - Pipeline pipeline = TestPipeline.create(); PCollection>> input = pipeline .apply(Create.of(GBK_TESTABLE_KVS) @@ -113,7 +114,6 @@ public void testReshuffleAfterSessionsAndGroupByKey() { @Test @Category(RunnableOnService.class) public void testReshuffleAfterFixedWindowsAndGroupByKey() { - Pipeline pipeline = TestPipeline.create(); PCollection>> input = pipeline .apply(Create.of(GBK_TESTABLE_KVS) @@ -137,7 +137,6 @@ public void testReshuffleAfterFixedWindowsAndGroupByKey() { @Test @Category(RunnableOnService.class) public void testReshuffleAfterSlidingWindowsAndGroupByKey() { - Pipeline pipeline = TestPipeline.create(); PCollection>> input = pipeline .apply(Create.of(GBK_TESTABLE_KVS) @@ -161,7 +160,6 @@ public void testReshuffleAfterSlidingWindowsAndGroupByKey() { @Test @Category(RunnableOnService.class) public void testReshuffleAfterFixedWindows() { - Pipeline pipeline = TestPipeline.create(); PCollection> input = pipeline .apply(Create.of(ARBITRARY_KVS) @@ -185,7 +183,6 @@ public void testReshuffleAfterFixedWindows() { @Test @Category(RunnableOnService.class) public void testReshuffleAfterSlidingWindows() { - Pipeline pipeline = TestPipeline.create(); PCollection> input = pipeline .apply(Create.of(ARBITRARY_KVS) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java index 1467ae8a52fb..b5351daac4e2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java @@ -24,7 +24,6 @@ import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; @@ -33,6 +32,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -41,9 +41,14 @@ /** Unit tests for {@link PCollectionTuple}. */ @RunWith(JUnit4.class) public final class PCollectionTupleTest implements Serializable { + + @Rule + public final transient TestPipeline pipeline = TestPipeline.create() + .enableAbandonedNodeEnforcement(false); + @Test public void testOfThenHas() { - Pipeline pipeline = TestPipeline.create(); + PCollection pCollection = PCollection.createPrimitiveOutputInternal( pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); TupleTag tag = new TupleTag<>(); @@ -53,7 +58,6 @@ public void testOfThenHas() { @Test public void testEmpty() { - Pipeline pipeline = TestPipeline.create(); TupleTag tag = new TupleTag<>(); assertFalse(PCollectionTuple.empty(pipeline).has(tag)); } @@ -61,7 +65,7 @@ public void testEmpty() { @Test @Category(RunnableOnService.class) public void testComposePCollectionTuple() { - Pipeline pipeline = TestPipeline.create(); + pipeline.enableAbandonedNodeEnforcement(true); List inputs = Arrays.asList(3, -42, 666); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java index e5f2019a19e9..ba7477da86a2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PDoneTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.TestUtils.LINES; import java.io.File; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.RunnableOnService; @@ -40,6 +39,10 @@ */ @RunWith(JUnit4.class) public class PDoneTest { + + @Rule + public final TestPipeline p = TestPipeline.create(); + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -78,8 +81,6 @@ public PDone expand(PBegin begin) { @Test @Category(RunnableOnService.class) public void testEmptyTransform() { - Pipeline p = TestPipeline.create(); - p.begin().apply(new EmptyTransform()); p.run(); @@ -94,8 +95,6 @@ public void testSimpleTransform() throws Exception { File tmpFile = tmpFolder.newFile("file.txt"); String filename = tmpFile.getPath(); - Pipeline p = TestPipeline.create(); - p.begin().apply(new SimpleTransform(filename)); p.run(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java index f33b3a2b692d..8381f120ceae 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypedPValueTest.java @@ -22,7 +22,6 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -40,6 +39,10 @@ */ @RunWith(JUnit4.class) public class TypedPValueTest { + + @Rule + public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public ExpectedException thrown = ExpectedException.none(); @@ -51,9 +54,8 @@ public void processElement(ProcessContext c) throws Exception { } } - private static PCollectionTuple buildPCollectionTupleWithTags( + private PCollectionTuple buildPCollectionTupleWithTags( TupleTag mainOutputTag, TupleTag sideOutputTag) { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(1, 2, 3)); PCollectionTuple tuple = input.apply( ParDo @@ -138,7 +140,6 @@ public void processElement(ProcessContext c) throws Exception { @Test public void testParDoWithNoSideOutputsErrorDoesNotMentionTupleTag() { - Pipeline p = TestPipeline.create(); PCollection input = p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new EmptyClassDoFn())); @@ -158,7 +159,6 @@ public void testParDoWithNoSideOutputsErrorDoesNotMentionTupleTag() { @Test public void testFinishSpecifyingShouldFailIfNoCoderInferrable() { - Pipeline p = TestPipeline.create(); PCollection unencodable = p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new EmptyClassDoFn())); From 24ad18319248a128a1c5db4f2bef8861f7361d9f Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 00:01:31 +0200 Subject: [PATCH 247/279] Migrated the beam-sdks-java-extensions-join-library module to TestPipeline as a JUnit rule. --- .../beam/sdk/extensions/joinlibrary/InnerJoinTest.java | 9 ++++++--- .../sdk/extensions/joinlibrary/OuterLeftJoinTest.java | 10 +++++++--- .../sdk/extensions/joinlibrary/OuterRightJoinTest.java | 10 +++++++--- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java index 423ab9c5cdcb..1c120c21faa7 100644 --- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/InnerJoinTest.java @@ -19,13 +19,13 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; /** @@ -33,15 +33,16 @@ */ public class InnerJoinTest { - Pipeline p; List> leftListOfKv; List> listRightOfKv; List>> expectedResult; + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Before public void setup() { - p = TestPipeline.create(); leftListOfKv = new ArrayList<>(); listRightOfKv = new ArrayList<>(); @@ -131,11 +132,13 @@ public void testJoinNoneToNoneMapping() { @Test(expected = NullPointerException.class) public void testJoinLeftCollectionNull() { + p.enableAbandonedNodeEnforcement(false); Join.innerJoin(null, p.apply(Create.of(listRightOfKv))); } @Test(expected = NullPointerException.class) public void testJoinRightCollectionNull() { + p.enableAbandonedNodeEnforcement(false); Join.innerJoin(p.apply(Create.of(leftListOfKv)), null); } } diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java index c32163fd5580..81f4fa3231fc 100644 --- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterLeftJoinTest.java @@ -19,13 +19,13 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; @@ -34,15 +34,16 @@ */ public class OuterLeftJoinTest { - Pipeline p; List> leftListOfKv; List> listRightOfKv; List>> expectedResult; + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Before public void setup() { - p = TestPipeline.create(); leftListOfKv = new ArrayList<>(); listRightOfKv = new ArrayList<>(); @@ -133,16 +134,19 @@ public void testJoinOneToNoneMapping() { @Test(expected = NullPointerException.class) public void testJoinLeftCollectionNull() { + p.enableAbandonedNodeEnforcement(false); Join.leftOuterJoin(null, p.apply(Create.of(listRightOfKv)), ""); } @Test(expected = NullPointerException.class) public void testJoinRightCollectionNull() { + p.enableAbandonedNodeEnforcement(false); Join.leftOuterJoin(p.apply(Create.of(leftListOfKv)), null, ""); } @Test(expected = NullPointerException.class) public void testJoinNullValueIsNull() { + p.enableAbandonedNodeEnforcement(false); Join.leftOuterJoin( p.apply("CreateLeft", Create.of(leftListOfKv)), p.apply("CreateRight", Create.of(listRightOfKv)), diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java index 5a45f73e59e7..249cea32e2e2 100644 --- a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterRightJoinTest.java @@ -19,13 +19,13 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; @@ -34,15 +34,16 @@ */ public class OuterRightJoinTest { - Pipeline p; List> leftListOfKv; List> listRightOfKv; List>> expectedResult; + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Before public void setup() { - p = TestPipeline.create(); leftListOfKv = new ArrayList<>(); listRightOfKv = new ArrayList<>(); @@ -133,16 +134,19 @@ public void testJoinNoneToOneMapping() { @Test(expected = NullPointerException.class) public void testJoinLeftCollectionNull() { + p.enableAbandonedNodeEnforcement(false); Join.rightOuterJoin(null, p.apply(Create.of(listRightOfKv)), ""); } @Test(expected = NullPointerException.class) public void testJoinRightCollectionNull() { + p.enableAbandonedNodeEnforcement(false); Join.rightOuterJoin(p.apply(Create.of(leftListOfKv)), null, -1L); } @Test(expected = NullPointerException.class) public void testJoinNullValueIsNull() { + p.enableAbandonedNodeEnforcement(false); Join.rightOuterJoin( p.apply("CreateLeft", Create.of(leftListOfKv)), p.apply("CreateRight", Create.of(listRightOfKv)), From 63331aa8aa6314e8469c23a4f4a89fbf287cbc5a Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 09:54:57 +0200 Subject: [PATCH 248/279] Migrated the beam-sdks-java-extensions-sorter module to TestPipeline as a JUnit rule. --- .../apache/beam/sdk/extensions/sorter/SortValuesTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java index ebfbd0e40fd1..4f771005322e 100644 --- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java +++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/SortValuesTest.java @@ -24,7 +24,6 @@ import static org.junit.Assert.assertThat; import java.util.Arrays; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -35,6 +34,7 @@ import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -43,10 +43,11 @@ @RunWith(JUnit4.class) public class SortValuesTest { + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test public void testSecondaryKeySorting() throws Exception { - Pipeline p = TestPipeline.create(); - // Create a PCollection of > pairs. PCollection>> input = p.apply( From 6dea0992d9976b39232cf846906831feaa25ec43 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 13:26:07 +0200 Subject: [PATCH 249/279] Migrated the beam-sdks-java-io-google-cloud-platform module to TestPipeline as a JUnit rule. --- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 25 +++++++++++++------ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 13 ++++++---- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index dc566d2c8634..4ddfdea1c704 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -643,6 +643,7 @@ public long insertAll( } } + @Rule public final transient TestPipeline p = TestPipeline.create(); @Rule public transient ExpectedException thrown = ExpectedException.none(); @Rule public transient ExpectedLogs logged = ExpectedLogs.none(BigQueryIO.class); @Rule public transient TemporaryFolder testFolder = new TemporaryFolder(); @@ -1370,7 +1371,7 @@ public void testBuildWriteWithTableReference() { @Test @Category(NeedsRunner.class) public void testBuildWriteWithoutTable() { - Pipeline p = TestPipeline.create(); + thrown.expect(IllegalStateException.class); thrown.expectMessage("must set the table reference"); p.apply(Create.of().withCoder(TableRowJsonCoder.of())) @@ -1591,9 +1592,11 @@ public void testBigQueryIOGetName() { @Test public void testWriteValidateFailsCreateNoSchema() { + p.enableAbandonedNodeEnforcement(false); + thrown.expect(IllegalArgumentException.class); thrown.expectMessage("no schema was provided"); - TestPipeline.create() + p .apply(Create.of()) .apply(BigQueryIO.Write .to("dataset.table") @@ -1602,9 +1605,11 @@ public void testWriteValidateFailsCreateNoSchema() { @Test public void testWriteValidateFailsTableAndTableSpec() { + p.enableAbandonedNodeEnforcement(false); + thrown.expect(IllegalStateException.class); thrown.expectMessage("Cannot set both a table reference and a table function"); - TestPipeline.create() + p .apply(Create.of()) .apply(BigQueryIO.Write .to("dataset.table") @@ -1618,9 +1623,11 @@ public String apply(BoundedWindow input) { @Test public void testWriteValidateFailsNoTableAndNoTableSpec() { + p.enableAbandonedNodeEnforcement(false); + thrown.expect(IllegalStateException.class); thrown.expectMessage("must set the table reference of a BigQueryIO.Write transform"); - TestPipeline.create() + p .apply(Create.of()) .apply("name", BigQueryIO.Write.withoutValidation()); } @@ -1950,7 +1957,6 @@ public String apply(Long input) { @Test @Category(NeedsRunner.class) public void testPassThroughThenCleanup() throws Exception { - Pipeline p = TestPipeline.create(); PCollection output = p .apply(Create.of(1, 2, 3)) @@ -1968,7 +1974,6 @@ void cleanup(PipelineOptions options) throws Exception { @Test @Category(NeedsRunner.class) public void testPassThroughThenCleanupExecuted() throws Exception { - Pipeline p = TestPipeline.create(); p.apply(Create.of()) .apply(new PassThroughThenCleanup(new CleanupOperation() { @@ -2025,6 +2030,8 @@ public void testWritePartitionLargeFileSize() throws Exception { private void testWritePartition(long numFiles, long fileSize, long expectedNumPartitions) throws Exception { + p.enableAbandonedNodeEnforcement(false); + List expectedPartitionIds = Lists.newArrayList(); for (long i = 1; i <= expectedNumPartitions; ++i) { expectedPartitionIds.add(i); @@ -2044,7 +2051,7 @@ private void testWritePartition(long numFiles, long fileSize, long expectedNumPa new TupleTag>>("singlePartitionTag") {}; PCollectionView>> filesView = PCollectionViews.iterableView( - TestPipeline.create(), + p, WindowingStrategy.globalDefault(), KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); @@ -2164,6 +2171,8 @@ public void testRemoveTemporaryFiles() throws Exception { @Test public void testWriteRename() throws Exception { + p.enableAbandonedNodeEnforcement(false); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done") @@ -2179,7 +2188,7 @@ public void testWriteRename() throws Exception { } PCollectionView> tempTablesView = PCollectionViews.iterableView( - TestPipeline.create(), + p, WindowingStrategy.globalDefault(), StringUtf8Coder.of()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 98215df019a3..e3d1f47c02c1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -106,6 +106,7 @@ */ @RunWith(JUnit4.class) public class BigtableIOTest { + @Rule public final transient TestPipeline p = TestPipeline.create(); @Rule public ExpectedException thrown = ExpectedException.none(); @Rule public ExpectedLogs logged = ExpectedLogs.none(BigtableIO.class); @@ -140,7 +141,7 @@ public void setup() throws Exception { service = new FakeBigtableService(); defaultRead = defaultRead.withBigtableService(service); defaultWrite = defaultWrite.withBigtableService(service); - bigtableCoder = TestPipeline.create().getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE); + bigtableCoder = p.getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE); } @Test @@ -261,6 +262,8 @@ public void testDontUsePipelineOptionsCredentialsIfSpecifiedInBigtableOptions() /** Tests that when reading from a non-existent table, the read fails. */ @Test public void testReadingFailsTableDoesNotExist() throws Exception { + p.enableAbandonedNodeEnforcement(false); + final String table = "TEST-TABLE"; BigtableIO.Read read = @@ -273,7 +276,7 @@ public void testReadingFailsTableDoesNotExist() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage(String.format("Table %s does not exist", table)); - TestPipeline.create().apply(read); + p.apply(read); } /** Tests that when reading from an empty table, the read succeeds. */ @@ -589,7 +592,6 @@ public void testWriting() throws Exception { service.createTable(table); - TestPipeline p = TestPipeline.create(); p.apply("single row", Create.of(makeWrite(key, value)).withCoder(bigtableCoder)) .apply("write", defaultWrite.withTableId(table)); p.run(); @@ -606,10 +608,12 @@ public void testWriting() throws Exception { /** Tests that when writing to a non-existent table, the write fails. */ @Test public void testWritingFailsTableDoesNotExist() throws Exception { + p.enableAbandonedNodeEnforcement(false); + final String table = "TEST-TABLE"; PCollection>> emptyInput = - TestPipeline.create().apply(Create.>>of()); + p.apply(Create.>>of()); // Exception will be thrown by write.validate() when write is applied. thrown.expect(IllegalArgumentException.class); @@ -625,7 +629,6 @@ public void testWritingFailsBadElement() throws Exception { final String key = "KEY"; service.createTable(table); - TestPipeline p = TestPipeline.create(); p.apply(Create.of(makeBadWrite(key)).withCoder(bigtableCoder)) .apply(defaultWrite.withTableId(table)); From 5ccbe6791af7e75c166ed877391e8c86bba5fe56 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 17:26:51 +0200 Subject: [PATCH 250/279] Migrated the beam-sdks-java-io-jdbc module to TestPipeline as a JUnit rule. --- .../test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java index aa93a222b6bc..eec7cb856cb9 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java @@ -48,6 +48,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -64,6 +65,9 @@ public class JdbcIOTest implements Serializable { private static int port; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @BeforeClass public static void startDatabase() throws Exception { ServerSocket socket = new ServerSocket(0); @@ -207,7 +211,6 @@ public void testDataSourceConfigurationNullUsernameAndPassword() throws Exceptio @Test @Category(NeedsRunner.class) public void testRead() throws Exception { - TestPipeline pipeline = TestPipeline.create(); PCollection> output = pipeline.apply( JdbcIO.>read() @@ -245,7 +248,6 @@ public Void apply(Iterable> input) { @Test @Category(NeedsRunner.class) public void testReadWithSingleStringParameter() throws Exception { - TestPipeline pipeline = TestPipeline.create(); PCollection> output = pipeline.apply( JdbcIO.>read() @@ -278,7 +280,6 @@ public KV mapRow(ResultSet resultSet) throws Exception { @Test @Category(NeedsRunner.class) public void testWrite() throws Exception { - TestPipeline pipeline = TestPipeline.create(); ArrayList> data = new ArrayList<>(); for (int i = 0; i < 1000; i++) { @@ -316,7 +317,6 @@ public void setParameters(KV element, PreparedStatement stateme @Test @Category(NeedsRunner.class) public void testWriteWithEmptyPCollection() throws Exception { - TestPipeline pipeline = TestPipeline.create(); pipeline.apply(Create.of(new ArrayList>())) .apply(JdbcIO.>write() From 8d478c0f38c656d3533d590a65c6ed95da229f81 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 17:31:23 +0200 Subject: [PATCH 251/279] Migrated the beam-sdks-java-io-jms module to TestPipeline as a JUnit rule. --- .../test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java index 4c3be6d0bae2..7259ce88b08d 100644 --- a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java +++ b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java @@ -28,7 +28,6 @@ import org.apache.activemq.ActiveMQConnectionFactory; import org.apache.activemq.broker.BrokerService; import org.apache.activemq.store.memory.MemoryPersistenceAdapter; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -38,6 +37,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -55,6 +55,9 @@ public class JmsIOTest { private BrokerService broker; private ConnectionFactory connectionFactory; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Before public void startBroker() throws Exception { broker = new BrokerService(); @@ -92,8 +95,6 @@ public void testReadMessages() throws Exception { session.close(); connection.close(); - Pipeline pipeline = TestPipeline.create(); - // read from the queue PCollection output = pipeline.apply( JmsIO.read() @@ -117,8 +118,6 @@ public void testReadMessages() throws Exception { @Category(NeedsRunner.class) public void testWriteMessage() throws Exception { - Pipeline pipeline = TestPipeline.create(); - ArrayList data = new ArrayList<>(); for (int i = 0; i < 100; i++) { data.add("Message " + i); From 12be8b1e6adf342e2b482aa37d5a9577e13802c5 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 17:38:38 +0200 Subject: [PATCH 252/279] Migrated the beam-sdks-java-io-kafka module to TestPipeline as a JUnit rule. --- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index cc1ef26ca919..071deea2f6bb 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -94,6 +93,9 @@ public class KafkaIOTest { * - test KafkaRecordCoder */ + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); @@ -268,7 +270,6 @@ public static void addCountingAsserts(PCollection input, long numElements) @Test @Category(NeedsRunner.class) public void testUnboundedSource() { - Pipeline p = TestPipeline.create(); int numElements = 1000; PCollection input = p @@ -283,7 +284,6 @@ public void testUnboundedSource() { @Test @Category(NeedsRunner.class) public void testUnboundedSourceWithExplicitPartitions() { - Pipeline p = TestPipeline.create(); int numElements = 1000; List topics = ImmutableList.of("test"); @@ -322,7 +322,7 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(NeedsRunner.class) public void testUnboundedSourceTimestamps() { - Pipeline p = TestPipeline.create(); + int numElements = 1000; PCollection input = p @@ -350,7 +350,7 @@ public void processElement(ProcessContext ctx) throws Exception { @Test @Category(NeedsRunner.class) public void testUnboundedSourceSplits() throws Exception { - Pipeline p = TestPipeline.create(); + int numElements = 1000; int numSplits = 10; @@ -514,10 +514,9 @@ public void testSink() throws Exception { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread().start(); - Pipeline pipeline = TestPipeline.create(); String topic = "test"; - pipeline + p .apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()) .withoutMetadata()) .apply(KafkaIO.write() @@ -527,7 +526,7 @@ public void testSink() throws Exception { .withValueCoder(BigEndianLongCoder.of()) .withProducerFactoryFn(new ProducerFactoryFn())); - pipeline.run(); + p.run(); completionThread.shutdown(); @@ -547,10 +546,9 @@ public void testValuesSink() throws Exception { ProducerSendCompletionThread completionThread = new ProducerSendCompletionThread().start(); - Pipeline pipeline = TestPipeline.create(); String topic = "test"; - pipeline + p .apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()) .withoutMetadata()) .apply(Values.create()) // there are no keys @@ -562,7 +560,7 @@ public void testValuesSink() throws Exception { .withProducerFactoryFn(new ProducerFactoryFn()) .values()); - pipeline.run(); + p.run(); completionThread.shutdown(); @@ -588,13 +586,12 @@ public void testSinkWithSendErrors() throws Throwable { MOCK_PRODUCER.clear(); - Pipeline pipeline = TestPipeline.create(); String topic = "test"; ProducerSendCompletionThread completionThreadWithErrors = new ProducerSendCompletionThread(10, 100).start(); - pipeline + p .apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()) .withoutMetadata()) .apply(KafkaIO.write() @@ -605,7 +602,7 @@ public void testSinkWithSendErrors() throws Throwable { .withProducerFactoryFn(new ProducerFactoryFn())); try { - pipeline.run(); + p.run(); } catch (PipelineExecutionException e) { // throwing inner exception helps assert that first exception is thrown from the Sink throw e.getCause().getCause(); From 950aa7e1d9c50167933eb192a16e15700e483377 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 17:44:15 +0200 Subject: [PATCH 253/279] Migrated the beam-sdks-java-io-kinesis module to TestPipeline as a JUnit rule. --- .../apache/beam/sdk/io/kinesis/KinesisMockReadTest.java | 7 +++++-- .../org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java | 6 ++++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java index f0ab46c3ae70..075805ef2d16 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java @@ -22,19 +22,23 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; import com.google.common.collect.Iterables; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.joda.time.DateTime; +import org.junit.Rule; import org.junit.Test; /** * Tests {@link AmazonKinesisMock}. */ public class KinesisMockReadTest { + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test public void readsDataFromMockKinesis() { int noOfShards = 3; @@ -42,7 +46,6 @@ public void readsDataFromMockKinesis() { List> testData = provideTestData(noOfShards, noOfEventsPerShard); - final Pipeline p = TestPipeline.create(); PCollection result = p. apply( KinesisIO.Read. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java index 73a24555fa9c..690cc11eab15 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java @@ -31,7 +31,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; @@ -43,6 +42,7 @@ import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Ignore; +import org.junit.Rule; import org.junit.Test; /** @@ -53,6 +53,8 @@ public class KinesisReaderIT { private static final long PIPELINE_STARTUP_TIME = TimeUnit.SECONDS.toMillis(10); private ExecutorService singleThreadExecutor = newSingleThreadExecutor(); + @Rule + public final transient TestPipeline p = TestPipeline.create(); @Ignore @Test @@ -76,7 +78,7 @@ private List prepareTestData(int count) { private Future startTestPipeline(List testData, KinesisTestOptions options) throws InterruptedException { - final Pipeline p = TestPipeline.create(); + PCollection result = p. apply(KinesisIO.Read. from(options.getAwsKinesisStream(), Instant.now()). From fce4f6584ca2fd3c2c258405b9f3014be9da3514 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 18:09:30 +0200 Subject: [PATCH 254/279] Migrated the beam-sdks-java-io-mongodb module to TestPipeline as a JUnit rule. --- .../apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java | 9 ++++----- .../org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java | 7 ++++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java index df05c934194c..994be87a6d64 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java @@ -55,7 +55,6 @@ import java.util.Random; import java.util.Scanner; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -79,6 +78,7 @@ import org.joda.time.Instant; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -100,6 +100,9 @@ public class MongoDBGridFSIOTest implements Serializable { private static int port; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @BeforeClass public static void setup() throws Exception { try (ServerSocket serverSocket = new ServerSocket(0)) { @@ -182,7 +185,6 @@ public static void stop() throws Exception { @Test @Category(NeedsRunner.class) public void testFullRead() throws Exception { - TestPipeline pipeline = TestPipeline.create(); PCollection output = pipeline.apply( MongoDbGridFSIO.read() @@ -212,7 +214,6 @@ public Void apply(Iterable> input) { @Test @Category(NeedsRunner.class) public void testReadWithParser() throws Exception { - TestPipeline pipeline = TestPipeline.create(); PCollection> output = pipeline.apply( MongoDbGridFSIO.>read() @@ -297,8 +298,6 @@ public void testSplit() throws Exception { @Category(NeedsRunner.class) public void testWriteMessage() throws Exception { - Pipeline pipeline = TestPipeline.create(); - ArrayList data = new ArrayList<>(100); ArrayList intData = new ArrayList<>(100); for (int i = 0; i < 1000; i++) { diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java index 5faa618ba75d..e7ff712e0b4c 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -57,6 +57,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -80,6 +81,9 @@ public class MongoDbIOTest implements Serializable { private static int port; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + /** * Looking for an available network port. */ @@ -143,7 +147,6 @@ public void stop() throws Exception { @Test @Category(NeedsRunner.class) public void testFullRead() throws Exception { - TestPipeline pipeline = TestPipeline.create(); PCollection output = pipeline.apply( MongoDbIO.read() @@ -177,7 +180,6 @@ public Void apply(Iterable> input) { @Test @Category(NeedsRunner.class) public void testReadWithFilter() throws Exception { - TestPipeline pipeline = TestPipeline.create(); PCollection output = pipeline.apply( MongoDbIO.read() @@ -195,7 +197,6 @@ public void testReadWithFilter() throws Exception { @Test @Category(NeedsRunner.class) public void testWrite() throws Exception { - TestPipeline pipeline = TestPipeline.create(); ArrayList data = new ArrayList<>(); for (int i = 0; i < 10000; i++) { From 4b23d42c31c95bed0d64bfc393fa193311e93498 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 20 Dec 2016 18:57:57 +0200 Subject: [PATCH 255/279] Migrated the beam-sdks-java-io-java8tests module to TestPipeline as a JUnit rule + fixed WithTimestampsJava8Test.withTimestampsLambdaShouldApplyTimestamps. --- .../beam/sdk/transforms/CombineJava8Test.java | 8 +++----- .../beam/sdk/transforms/DistinctJava8Test.java | 5 +++-- .../beam/sdk/transforms/FilterJava8Test.java | 9 +++------ .../sdk/transforms/FlatMapElementsJava8Test.java | 7 ++++--- .../beam/sdk/transforms/MapElementsJava8Test.java | 9 ++++++--- .../beam/sdk/transforms/PartitionJava8Test.java | 7 ++++--- .../beam/sdk/transforms/WithKeysJava8Test.java | 6 ++++-- .../sdk/transforms/WithTimestampsJava8Test.java | 14 ++++++++++---- 8 files changed, 37 insertions(+), 28 deletions(-) diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java index 98d99cefd93d..a0f7ce65f87a 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/CombineJava8Test.java @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -44,6 +43,9 @@ @SuppressWarnings("serial") public class CombineJava8Test implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -65,7 +67,6 @@ public int sum(Iterable integers) { */ @Test public void testCombineGloballyLambda() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3, 4)) @@ -86,7 +87,6 @@ public void testCombineGloballyLambda() { */ @Test public void testCombineGloballyInstanceMethodReference() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3, 4)) @@ -101,7 +101,6 @@ public void testCombineGloballyInstanceMethodReference() { */ @Test public void testCombinePerKeyLambda() { - Pipeline pipeline = TestPipeline.create(); PCollection> output = pipeline .apply(Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 3), KV.of("c", 4))) @@ -125,7 +124,6 @@ public void testCombinePerKeyLambda() { */ @Test public void testCombinePerKeyInstanceMethodReference() { - Pipeline pipeline = TestPipeline.create(); PCollection> output = pipeline .apply(Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 3), KV.of("c", 4))) diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java index 99ef23289457..790f51e06c91 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/DistinctJava8Test.java @@ -43,12 +43,14 @@ @RunWith(JUnit4.class) public class DistinctJava8Test { + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); @Test public void withLambdaRepresentativeValuesFnAndTypeDescriptorShouldApplyFn() { - TestPipeline p = TestPipeline.create(); Multimap predupedContents = HashMultimap.create(); predupedContents.put(3, "foo"); @@ -76,7 +78,6 @@ public void withLambdaRepresentativeValuesFnAndTypeDescriptorShouldApplyFn() { @Test public void withLambdaRepresentativeValuesFnNoTypeDescriptorShouldThrow() { - TestPipeline p = TestPipeline.create(); Multimap predupedContents = HashMultimap.create(); predupedContents.put(3, "foo"); diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java index afd1c8b48ca5..f91371ee097d 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FilterJava8Test.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.transforms; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -38,13 +37,15 @@ @SuppressWarnings("serial") public class FilterJava8Test implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test @Category(RunnableOnService.class) public void testIdentityFilterByPredicate() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(591, 11789, 1257, 24578, 24799, 307)) @@ -56,7 +57,6 @@ public void testIdentityFilterByPredicate() { @Test public void testNoFilterByPredicate() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 4, 5)) @@ -69,7 +69,6 @@ public void testNoFilterByPredicate() { @Test @Category(RunnableOnService.class) public void testFilterByPredicate() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) @@ -85,7 +84,6 @@ public void testFilterByPredicate() { */ @Test public void testFilterParDoOutputTypeDescriptorRaw() throws Exception { - Pipeline pipeline = TestPipeline.create(); @SuppressWarnings({"unchecked", "rawtypes"}) PCollection output = pipeline @@ -99,7 +97,6 @@ public void testFilterParDoOutputTypeDescriptorRaw() throws Exception { @Test @Category(RunnableOnService.class) public void testFilterByMethodReference() { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java index 70cc04da9aff..471724db41bf 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsJava8Test.java @@ -20,7 +20,6 @@ import com.google.common.collect.ImmutableList; import java.io.Serializable; import java.util.List; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; @@ -37,6 +36,9 @@ @RunWith(JUnit4.class) public class FlatMapElementsJava8Test implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -46,7 +48,6 @@ public class FlatMapElementsJava8Test implements Serializable { */ @Test public void testFlatMapBasic() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline .apply(Create.of(1, 2, 3)) .apply(FlatMapElements @@ -63,7 +64,7 @@ public void testFlatMapBasic() throws Exception { */ @Test public void testFlatMapMethodReference() throws Exception { - Pipeline pipeline = TestPipeline.create(); + PCollection output = pipeline .apply(Create.of(1, 2, 3)) .apply(FlatMapElements diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java index 9b556b955686..ce0f1116c6c3 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/MapElementsJava8Test.java @@ -18,11 +18,11 @@ package org.apache.beam.sdk.transforms; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,13 +33,16 @@ @RunWith(JUnit4.class) public class MapElementsJava8Test implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + /** * Basic test of {@link MapElements} with a lambda (which is instantiated as a * {@link SerializableFunction}). */ @Test public void testMapBasic() throws Exception { - Pipeline pipeline = TestPipeline.create(); + PCollection output = pipeline .apply(Create.of(1, 2, 3)) .apply(MapElements @@ -56,7 +59,7 @@ public void testMapBasic() throws Exception { */ @Test public void testMapMethodReference() throws Exception { - Pipeline pipeline = TestPipeline.create(); + PCollection output = pipeline .apply(Create.of(1, 2, 3)) .apply(MapElements diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java index 0aeb41ffdc92..7d977402dd54 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/PartitionJava8Test.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -38,12 +37,15 @@ @SuppressWarnings("serial") public class PartitionJava8Test implements Serializable { + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test public void testModPartition() { - Pipeline pipeline = TestPipeline.create(); + PCollectionList outputs = pipeline .apply(Create.of(1, 2, 4, 5)) @@ -61,7 +63,6 @@ public void testModPartition() { */ @Test public void testPartitionFnOutputTypeDescriptorRaw() throws Exception { - Pipeline pipeline = TestPipeline.create(); PCollectionList output = pipeline .apply(Create.of("hello")) diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java index a5b9cb1239a0..6ba41fac2b8b 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithKeysJava8Test.java @@ -38,13 +38,16 @@ @RunWith(JUnit4.class) public class WithKeysJava8Test { + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); @Test @Category(RunnableOnService.class) public void withLambdaAndTypeDescriptorShouldSucceed() { - TestPipeline p = TestPipeline.create(); + PCollection values = p.apply(Create.of("1234", "3210", "0", "-12")); PCollection> kvs = values.apply( @@ -59,7 +62,6 @@ public void withLambdaAndTypeDescriptorShouldSucceed() { @Test public void withLambdaAndNoTypeDescriptorShouldThrow() { - TestPipeline p = TestPipeline.create(); PCollection values = p.apply(Create.of("1234", "3210", "0", "-12")); diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java index 5f1e74bd2ddd..a0c63702cb0e 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsJava8Test.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -34,15 +35,18 @@ */ @RunWith(JUnit4.class) public class WithTimestampsJava8Test implements Serializable { + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + @Test @Category(RunnableOnService.class) public void withTimestampsLambdaShouldApplyTimestamps() { - TestPipeline p = TestPipeline.create(); - String yearTwoThousand = "946684800000"; + final String yearTwoThousand = "946684800000"; PCollection timestamped = p.apply(Create.of("1234", "0", Integer.toString(Integer.MAX_VALUE), yearTwoThousand)) - .apply(WithTimestamps.of((String input) -> new Instant(Long.valueOf(yearTwoThousand)))); + .apply(WithTimestamps.of((String input) -> new Instant(Long.valueOf(input)))); PCollection> timestampedVals = timestamped.apply(ParDo.of(new DoFn>() { @@ -58,8 +62,10 @@ public void processElement(ProcessContext c) PAssert.that(timestampedVals) .containsInAnyOrder( KV.of("0", new Instant(0)), - KV.of("1234", new Instant("1234")), + KV.of("1234", new Instant(Long.valueOf("1234"))), KV.of(Integer.toString(Integer.MAX_VALUE), new Instant(Integer.MAX_VALUE)), KV.of(yearTwoThousand, new Instant(Long.valueOf(yearTwoThousand)))); + + p.run(); } } From 81702e67b92a23849cbc8f4a16b2a619e4b477a1 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 8 Dec 2016 11:49:15 -0800 Subject: [PATCH 256/279] Add some key-preserving to KeyedPValueTrackingVisitor --- .../beam/runners/direct/DirectRunner.java | 9 +- .../direct/KeyedPValueTrackingVisitor.java | 35 +++++--- .../KeyedPValueTrackingVisitorTest.java | 84 +++---------------- 3 files changed, 37 insertions(+), 91 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 78163c0d7e23..afa43ff3368a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -31,8 +31,6 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.SplittableParDo; -import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; -import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; import org.apache.beam.runners.direct.ViewEvaluatorFactory.ViewOverrideFactory; @@ -306,12 +304,7 @@ public DirectPipelineResult run(Pipeline pipeline) { graphVisitor.finishSpecifyingRemainder(); @SuppressWarnings("rawtypes") - KeyedPValueTrackingVisitor keyedPValueVisitor = - KeyedPValueTrackingVisitor.create( - ImmutableSet.of( - SplittableParDo.GBKIntoKeyedWorkItems.class, - DirectGroupByKeyOnly.class, - DirectGroupAlsoByWindow.class)); + KeyedPValueTrackingVisitor keyedPValueVisitor = KeyedPValueTrackingVisitor.create(); pipeline.traverseTopologically(keyedPValueVisitor); DisplayDataValidator.validatePipeline(pipeline); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 7f85169478a1..e91a768c4d7d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -18,9 +18,15 @@ package org.apache.beam.runners.direct; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Predicates.in; +import static com.google.common.collect.Iterables.all; +import com.google.common.collect.ImmutableSet; import java.util.HashSet; import java.util.Set; +import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; +import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.GroupByKey; @@ -38,19 +44,21 @@ // TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms // unkeyed class KeyedPValueTrackingVisitor implements PipelineVisitor { - @SuppressWarnings("rawtypes") - private final Set> producesKeyedOutputs; + + private static final Set> PRODUCES_KEYED_OUTPUTS = + ImmutableSet.of( + SplittableParDo.GBKIntoKeyedWorkItems.class, + DirectGroupByKeyOnly.class, + DirectGroupAlsoByWindow.class); + private final Set keyedValues; private boolean finalized; - public static KeyedPValueTrackingVisitor create( - @SuppressWarnings("rawtypes") Set> producesKeyedOutputs) { - return new KeyedPValueTrackingVisitor(producesKeyedOutputs); + public static KeyedPValueTrackingVisitor create() { + return new KeyedPValueTrackingVisitor(); } - private KeyedPValueTrackingVisitor( - @SuppressWarnings("rawtypes") Set> producesKeyedOutputs) { - this.producesKeyedOutputs = producesKeyedOutputs; + private KeyedPValueTrackingVisitor() { this.keyedValues = new HashSet<>(); } @@ -73,7 +81,7 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { node); if (node.isRootNode()) { finalized = true; - } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) { + } else if (PRODUCES_KEYED_OUTPUTS.contains(node.getTransform().getClass())) { keyedValues.addAll(node.getOutputs()); } } @@ -83,7 +91,9 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) {} @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { - if (producesKeyedOutputs.contains(producer.getTransform().getClass())) { + if (PRODUCES_KEYED_OUTPUTS.contains(producer.getTransform().getClass()) + || (isKeyPreserving(producer.getTransform()) + && all(producer.getInputs(), in(keyedValues)))) { keyedValues.add(value); } } @@ -93,4 +103,9 @@ public Set getKeyedPValues() { finalized, "can't call getKeyedPValues before a Pipeline has been completely traversed"); return keyedValues; } + + private static boolean isKeyPreserving(PTransform transform) { + // There are currently no key-preserving transforms; this lays the infrastructure for them + return false; + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index eef3375b445e..a35700503a36 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -21,9 +21,7 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; -import com.google.common.collect.ImmutableSet; import java.util.Collections; -import java.util.Set; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -33,7 +31,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.Keys; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -57,54 +54,20 @@ public class KeyedPValueTrackingVisitorTest { @Before public void setup() { - - @SuppressWarnings("rawtypes") - Set> producesKeyed = - ImmutableSet.>of(PrimitiveKeyer.class, CompositeKeyer.class); - visitor = KeyedPValueTrackingVisitor.create(producesKeyed); - } - - @Test - public void primitiveProducesKeyedOutputUnkeyedInputKeyedOutput() { - PCollection keyed = - p.apply(Create.of(1, 2, 3)).apply(new PrimitiveKeyer()); - - p.traverseTopologically(visitor); - assertThat(visitor.getKeyedPValues(), hasItem(keyed)); - } - - @Test - public void primitiveProducesKeyedOutputKeyedInputKeyedOutut() { - PCollection keyed = - p.apply(Create.of(1, 2, 3)) - .apply("firstKey", new PrimitiveKeyer()) - .apply("secondKey", new PrimitiveKeyer()); - - p.traverseTopologically(visitor); - assertThat(visitor.getKeyedPValues(), hasItem(keyed)); - } - - @Test - public void compositeProducesKeyedOutputUnkeyedInputKeyedOutput() { - PCollection keyed = - p.apply(Create.of(1, 2, 3)).apply(new CompositeKeyer()); - - p.traverseTopologically(visitor); - assertThat(visitor.getKeyedPValues(), hasItem(keyed)); + p = TestPipeline.create(); + visitor = KeyedPValueTrackingVisitor.create(); } @Test - public void compositeProducesKeyedOutputKeyedInputKeyedOutut() { - PCollection keyed = - p.apply(Create.of(1, 2, 3)) - .apply("firstKey", new CompositeKeyer()) - .apply("secondKey", new CompositeKeyer()); + public void groupByKeyProducesKeyedOutput() { + PCollection>> keyed = + p.apply(Create.of(KV.of("foo", 3))) + .apply(GroupByKey.create()); p.traverseTopologically(visitor); assertThat(visitor.getKeyedPValues(), hasItem(keyed)); } - @Test public void noInputUnkeyedOutput() { PCollection>> unkeyed = @@ -117,25 +80,16 @@ public void noInputUnkeyedOutput() { } @Test - public void keyedInputNotProducesKeyedOutputUnkeyedOutput() { - PCollection onceKeyed = - p.apply(Create.of(1, 2, 3)) - .apply(new PrimitiveKeyer()) - .apply(ParDo.of(new IdentityFn())); + public void keyedInputWithoutKeyPreserving() { + PCollection>> onceKeyed = + p.apply(Create.of(KV.of("hello", 42))) + .apply(GroupByKey.create()) + .apply(ParDo.of(new IdentityFn>>())); p.traverseTopologically(visitor); assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed))); } - @Test - public void unkeyedInputNotProducesKeyedOutputUnkeyedOutput() { - PCollection unkeyed = - p.apply(Create.of(1, 2, 3)).apply(ParDo.of(new IdentityFn())); - - p.traverseTopologically(visitor); - assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed))); - } - @Test public void traverseMultipleTimesThrows() { p.apply( @@ -161,22 +115,6 @@ public void getKeyedPValuesBeforeTraverseThrows() { visitor.getKeyedPValues(); } - private static class PrimitiveKeyer extends PTransform, PCollection> { - @Override - public PCollection expand(PCollection input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); - } - } - - private static class CompositeKeyer extends PTransform, PCollection> { - @Override - public PCollection expand(PCollection input) { - return input.apply(new PrimitiveKeyer()).apply(ParDo.of(new IdentityFn())); - } - } - private static class IdentityFn extends DoFn { @ProcessElement public void processElement(ProcessContext c) throws Exception { From b26ceaa347c4bc50abfb4c3c138167a25a99cf57 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 8 Dec 2016 13:28:44 -0800 Subject: [PATCH 257/279] Move responsibility for knowing about keyedness into EvaluationContext This will allow transform evaluators to inquire about whether various collections are keyed. --- .../beam/runners/direct/DirectRunner.java | 4 +-- .../runners/direct/EvaluationContext.java | 26 ++++++++++++++++--- .../ExecutorServiceParallelExecutor.java | 8 +----- .../runners/direct/EvaluationContextTest.java | 9 ++++++- 4 files changed, 34 insertions(+), 13 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index afa43ff3368a..7e6ea1556387 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -315,14 +315,14 @@ public DirectPipelineResult run(Pipeline pipeline) { getPipelineOptions(), clockSupplier.get(), Enforcement.bundleFactoryFor(enabledEnforcements, graph), - graph); + graph, + keyedPValueVisitor.getKeyedPValues()); RootProviderRegistry rootInputProvider = RootProviderRegistry.defaultRegistry(context); TransformEvaluatorRegistry registry = TransformEvaluatorRegistry.defaultRegistry(context); PipelineExecutor executor = ExecutorServiceParallelExecutor.create( options.getTargetParallelism(), graph, - keyedPValueVisitor.getKeyedPValues(), rootInputProvider, registry, Enforcement.defaultModelEnforcements(enabledEnforcements), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index 230d91b52dff..cb9ddd899800 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -27,6 +27,7 @@ import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; @@ -99,17 +100,28 @@ class EvaluationContext { private final DirectMetrics metrics; + private final Set keyedPValues; + public static EvaluationContext create( - DirectOptions options, Clock clock, BundleFactory bundleFactory, DirectGraph graph) { - return new EvaluationContext(options, clock, bundleFactory, graph); + DirectOptions options, + Clock clock, + BundleFactory bundleFactory, + DirectGraph graph, + Set keyedPValues) { + return new EvaluationContext(options, clock, bundleFactory, graph, keyedPValues); } private EvaluationContext( - DirectOptions options, Clock clock, BundleFactory bundleFactory, DirectGraph graph) { + DirectOptions options, + Clock clock, + BundleFactory bundleFactory, + DirectGraph graph, + Set keyedPValues) { this.options = checkNotNull(options); this.clock = clock; this.bundleFactory = checkNotNull(bundleFactory); this.graph = checkNotNull(graph); + this.keyedPValues = keyedPValues; this.watermarkManager = WatermarkManager.create(clock, graph); this.sideInputContainer = SideInputContainer.create(this, graph.getViews()); @@ -243,6 +255,14 @@ public UncommittedBundle createKeyedBundle( return bundleFactory.createKeyedBundle(key, output); } + /** + * Indicate whether or not this {@link PCollection} has been determined to be + * keyed. + */ + public boolean isKeyed(PValue pValue) { + return keyedPValues.contains(pValue); + } + /** * Create a {@link PCollectionViewWriter}, whose elements will be used in the provided * {@link PCollectionView}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index a30829520d9f..5a653b749687 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -31,7 +31,6 @@ import java.util.Collections; import java.util.Map; import java.util.Queue; -import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; @@ -70,7 +69,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor { private final ExecutorService executorService; private final DirectGraph graph; - private final Set keyedPValues; private final RootProviderRegistry rootProviderRegistry; private final TransformEvaluatorRegistry registry; @SuppressWarnings("rawtypes") @@ -105,7 +103,6 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor { public static ExecutorServiceParallelExecutor create( int targetParallelism, DirectGraph graph, - Set keyedPValues, RootProviderRegistry rootProviderRegistry, TransformEvaluatorRegistry registry, @SuppressWarnings("rawtypes") @@ -115,7 +112,6 @@ public static ExecutorServiceParallelExecutor create( return new ExecutorServiceParallelExecutor( targetParallelism, graph, - keyedPValues, rootProviderRegistry, registry, transformEnforcements, @@ -125,7 +121,6 @@ public static ExecutorServiceParallelExecutor create( private ExecutorServiceParallelExecutor( int targetParallelism, DirectGraph graph, - Set keyedPValues, RootProviderRegistry rootProviderRegistry, TransformEvaluatorRegistry registry, @SuppressWarnings("rawtypes") @@ -134,7 +129,6 @@ private ExecutorServiceParallelExecutor( this.targetParallelism = targetParallelism; this.executorService = Executors.newFixedThreadPool(targetParallelism); this.graph = graph; - this.keyedPValues = keyedPValues; this.rootProviderRegistry = rootProviderRegistry; this.registry = registry; this.transformEnforcements = transformEnforcements; @@ -229,7 +223,7 @@ private void evaluateBundle( } private boolean isKeyed(PValue pvalue) { - return keyedPValues.contains(pvalue); + return evaluationContext.isKeyed(pvalue); } private void scheduleConsumers(ExecutorUpdate update) { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index bf362048c8d8..15340da39ecc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -105,11 +105,18 @@ public void setup() { view = created.apply(View.asIterable()); unbounded = p.apply(CountingInput.unbounded()); + KeyedPValueTrackingVisitor keyedPValueTrackingVisitor = KeyedPValueTrackingVisitor.create(); + p.traverseTopologically(keyedPValueTrackingVisitor); + BundleFactory bundleFactory = ImmutableListBundleFactory.create(); graph = DirectGraphs.getGraph(p); context = EvaluationContext.create( - runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph); + runner.getPipelineOptions(), + NanosOffsetClock.create(), + bundleFactory, + graph, + keyedPValueTrackingVisitor.getKeyedPValues()); createdProducer = graph.getProducer(created); downstreamProducer = graph.getProducer(downstream); From d040b7f6a3cdefde829321015c75a800901cd88f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 8 Dec 2016 11:44:48 -0800 Subject: [PATCH 258/279] Propagate key through ParDo if DoFn is key-preserving --- .../apache/beam/runners/direct/ParDoEvaluator.java | 13 +++++++++++-- .../beam/runners/direct/ParDoEvaluatorFactory.java | 3 +++ .../SplittableProcessElementsEvaluatorFactory.java | 1 + .../beam/runners/direct/ParDoEvaluatorTest.java | 1 + 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index a915cf0bc990..a5de4c69941c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -47,6 +47,7 @@ public static ParDoEvaluator create( AppliedPTransform application, WindowingStrategy windowingStrategy, Serializable fn, // may be OldDoFn or DoFn + StructuralKey key, List> sideInputs, TupleTag mainOutputTag, List> sideOutputTags, @@ -55,8 +56,16 @@ public static ParDoEvaluator create( Map, UncommittedBundle> outputBundles = new HashMap<>(); for (Map.Entry, PCollection> outputEntry : outputs.entrySet()) { - outputBundles.put( - outputEntry.getKey(), evaluationContext.createBundle(outputEntry.getValue())); + // Just trust the context's decision as to whether the output should be keyed. + // The logic for whether this ParDo is key-preserving and whether the input + // is keyed lives elsewhere. + if (evaluationContext.isKeyed(outputEntry.getValue())) { + outputBundles.put( + outputEntry.getKey(), evaluationContext.createKeyedBundle(key, outputEntry.getValue())); + } else { + outputBundles.put( + outputEntry.getKey(), evaluationContext.createBundle(outputEntry.getValue())); + } } BundleOutputManager outputManager = BundleOutputManager.create(outputBundles); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index b4684e34b266..835e6ce6cff4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -112,6 +112,7 @@ TransformEvaluator createEvaluator( return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping( createParDoEvaluator( application, + inputBundleKey, sideInputs, mainOutputTag, sideOutputTags, @@ -123,6 +124,7 @@ TransformEvaluator createEvaluator( ParDoEvaluator createParDoEvaluator( AppliedPTransform, PCollectionTuple, ?> application, + StructuralKey key, List> sideInputs, TupleTag mainOutputTag, List> sideOutputTags, @@ -137,6 +139,7 @@ ParDoEvaluator createParDoEvaluator( application, application.getInput().getWindowingStrategy(), fn, + key, sideInputs, mainOutputTag, sideOutputTags, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index aae1149f99e1..18f390953c90 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -91,6 +91,7 @@ public void cleanup() throws Exception { parDoEvaluator = delegateFactory.createParDoEvaluator( application, + inputBundle.getKey(), transform.getSideInputs(), transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 1a3207b5b9f7..b3aceeb0635b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -164,6 +164,7 @@ private ParDoEvaluator createEvaluator( transform, transform.getInput().getWindowingStrategy(), fn, + null /* key */, ImmutableList.>of(singletonView), mainOutputTag, sideOutputTags, From 1f018ab69fdcc720a10e2aeb8ec1eea1c06e1cbc Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 12 Dec 2016 19:49:58 -0800 Subject: [PATCH 259/279] Port direct runner StatefulParDo to KeyedWorkItem --- .../direct/KeyedPValueTrackingVisitor.java | 13 ++- .../direct/ParDoMultiOverrideFactory.java | 94 ++++++++++++++++--- .../direct/StatefulParDoEvaluatorFactory.java | 36 +++---- .../KeyedPValueTrackingVisitorTest.java | 69 ++++++++++++-- .../StatefulParDoEvaluatorFactoryTest.java | 51 ++++++---- 5 files changed, 205 insertions(+), 58 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index e91a768c4d7d..65c41e06ba9f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.runners.TransformHierarchy; 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.values.PValue; /** @@ -105,7 +106,15 @@ public Set getKeyedPValues() { } private static boolean isKeyPreserving(PTransform transform) { - // There are currently no key-preserving transforms; this lays the infrastructure for them - return false; + // This is a hacky check for what is considered key-preserving to the direct runner. + // The most obvious alternative would be a package-private marker interface, but + // better to make this obviously hacky so it is less likely to proliferate. Meanwhile + // we intend to allow explicit expression of key-preserving DoFn in the model. + if (transform instanceof ParDo.BoundMulti) { + ParDo.BoundMulti parDo = (ParDo.BoundMulti) transform; + return parDo.getFn() instanceof ParDoMultiOverrideFactory.ToKeyedWorkItem; + } else { + return false; + } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index c5bc0698cad7..2cea9993ae44 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -17,9 +17,15 @@ */ package org.apache.beam.runners.direct; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItemCoder; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; @@ -28,6 +34,8 @@ import org.apache.beam.sdk.transforms.ParDo.BoundMulti; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -84,16 +92,41 @@ public GbkThenStatefulParDo(ParDo.BoundMulti, OutputT> underlyingP @Override public PCollectionTuple expand(PCollection> input) { - PCollectionTuple outputs = input - .apply("Group by key", GroupByKey.create()) - .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input)); + // A KvCoder is required since this goes through GBK. Further, WindowedValueCoder + // is not registered by default, so we explicitly set the relevant coders. + checkState(input.getCoder() instanceof KvCoder, + "Input to a %s using state requires a %s, but the coder was %s", + ParDo.class.getSimpleName(), + KvCoder.class.getSimpleName(), + input.getCoder()); + KvCoder kvCoder = (KvCoder) input.getCoder(); + Coder keyCoder = kvCoder.getKeyCoder(); + Coder windowCoder = + input.getWindowingStrategy().getWindowFn().windowCoder(); + + PCollectionTuple outputs = + input + // Stash the original timestamps, etc, for when it is fed to the user's DoFn + .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn())) + .setCoder(KvCoder.of(keyCoder, WindowedValue.getFullCoder(kvCoder, windowCoder))) + + // A full GBK to group by key _and_ window + .apply("Group by key", GroupByKey.>>create()) + + // Adapt to KeyedWorkItem; that is how this runner delivers timers + .apply("To KeyedWorkItem", ParDo.of(new ToKeyedWorkItem())) + .setCoder(KeyedWorkItemCoder.of(keyCoder, kvCoder, windowCoder)) + + // Explode the resulting iterable into elements that are exactly the ones from + // the input + .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input)); return outputs; } } static class StatefulParDo - extends PTransform>>, PCollectionTuple> { + extends PTransform>>, PCollectionTuple> { private final transient ParDo.BoundMulti, OutputT> underlyingParDo; private final transient PCollection> originalInput; @@ -110,21 +143,58 @@ public ParDo.BoundMulti, OutputT> getUnderlyingParDo() { @Override public Coder getDefaultOutputCoder( - PCollection>> input, TypedPValue output) + PCollection>> input, TypedPValue output) throws CannotProvideCoderException { return underlyingParDo.getDefaultOutputCoder(originalInput, output); } - public PCollectionTuple expand(PCollection>> input) { + @Override + public PCollectionTuple expand(PCollection>> input) { - PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( - input.getPipeline(), - TupleTagList.of(underlyingParDo.getMainOutputTag()) - .and(underlyingParDo.getSideOutputTags().getAll()), - input.getWindowingStrategy(), - input.isBounded()); + PCollectionTuple outputs = + PCollectionTuple.ofPrimitiveOutputsInternal( + input.getPipeline(), + TupleTagList.of(underlyingParDo.getMainOutputTag()) + .and(underlyingParDo.getSideOutputTags().getAll()), + input.getWindowingStrategy(), + input.isBounded()); return outputs; } } + + /** + * A distinguished key-preserving {@link DoFn}. + * + *

      This wraps the {@link GroupByKey} output in a {@link KeyedWorkItem} to be able to deliver + * timers. It also explodes them into single {@link KV KVs} since this is what the user's {@link + * DoFn} needs to process anyhow. + */ + static class ReifyWindowedValueFn extends DoFn, KV>>> { + @ProcessElement + public void processElement(final ProcessContext c, final BoundedWindow window) { + c.output( + KV.of( + c.element().getKey(), + WindowedValue.of(c.element(), c.timestamp(), window, c.pane()))); + } + } + + /** + * A runner-specific primitive that is just a key-preserving {@link ParDo}, but we do not have the + * machinery to detect or enforce that yet. + * + *

      This wraps the {@link GroupByKey} output in a {@link KeyedWorkItem} to be able to deliver + * timers. It also explodes them into single {@link KV KVs} since this is what the user's {@link + * DoFn} needs to process anyhow. + */ + static class ToKeyedWorkItem + extends DoFn>>>, KeyedWorkItem>> { + + @ProcessElement + public void processElement(final ProcessContext c, final BoundedWindow window) { + final K key = c.element().getKey(); + c.output(KeyedWorkItems.elementsWorkItem(key, c.element().getValue())); + } + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 1f64d9ac7059..5f9d8f49ed94 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -23,6 +23,8 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.Lists; import java.util.Collections; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; @@ -77,12 +79,12 @@ public void cleanup() throws Exception { } @SuppressWarnings({"unchecked", "rawtypes"}) - private TransformEvaluator>> createEvaluator( + private TransformEvaluator>> createEvaluator( AppliedPTransform< - PCollection>>, PCollectionTuple, + PCollection>>, PCollectionTuple, StatefulParDo> application, - CommittedBundle>> inputBundle) + CommittedBundle>> inputBundle) throws Exception { final DoFn, OutputT> doFn = @@ -185,7 +187,7 @@ public void run() { @AutoValue abstract static class AppliedPTransformOutputKeyAndWindow { abstract AppliedPTransform< - PCollection>>, PCollectionTuple, + PCollection>>, PCollectionTuple, StatefulParDo> getTransform(); @@ -195,7 +197,7 @@ abstract static class AppliedPTransformOutputKeyAndWindow { static AppliedPTransformOutputKeyAndWindow create( AppliedPTransform< - PCollection>>, PCollectionTuple, + PCollection>>, PCollectionTuple, StatefulParDo> transform, StructuralKey key, @@ -206,7 +208,7 @@ static AppliedPTransformOutputKeyAndWindow - implements TransformEvaluator>> { + implements TransformEvaluator>> { private final TransformEvaluator> delegateEvaluator; @@ -215,20 +217,20 @@ public StatefulParDoEvaluator(TransformEvaluator> delegateEvaluato } @Override - public void processElement(WindowedValue>> gbkResult) throws Exception { + public void processElement(WindowedValue>> gbkResult) + throws Exception { - for (InputT value : gbkResult.getValue().getValue()) { - delegateEvaluator.processElement( - gbkResult.withValue(KV.of(gbkResult.getValue().getKey(), value))); + for (WindowedValue> windowedValue : gbkResult.getValue().elementsIterable()) { + delegateEvaluator.processElement(windowedValue); } } @Override - public TransformResult>> finishBundle() throws Exception { + public TransformResult>> finishBundle() throws Exception { TransformResult> delegateResult = delegateEvaluator.finishBundle(); - StepTransformResult.Builder>> regroupedResult = - StepTransformResult.>>withHold( + StepTransformResult.Builder>> regroupedResult = + StepTransformResult.>>withHold( delegateResult.getTransform(), delegateResult.getWatermarkHold()) .withTimerUpdate(delegateResult.getTimerUpdate()) .withAggregatorChanges(delegateResult.getAggregatorChanges()) @@ -240,12 +242,10 @@ public TransformResult>> finishBundle() throws Exception // outputs, but just make a bunch of singletons for (WindowedValue untypedUnprocessed : delegateResult.getUnprocessedElements()) { WindowedValue> windowedKv = (WindowedValue>) untypedUnprocessed; - WindowedValue>> pushedBack = + WindowedValue>> pushedBack = windowedKv.withValue( - KV.of( - windowedKv.getValue().getKey(), - (Iterable) - Collections.singletonList(windowedKv.getValue().getValue()))); + KeyedWorkItems.elementsWorkItem( + windowedKv.getValue().getKey(), Collections.singleton(windowedKv))); regroupedResult.addUnprocessedElements(pushedBack); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index a35700503a36..a1fb81b39e93 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -22,8 +22,10 @@ import static org.junit.Assert.assertThat; import java.util.Collections; +import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.TestPipeline; @@ -32,8 +34,12 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.Keys; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -41,9 +47,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for {@link KeyedPValueTrackingVisitor}. - */ +/** Tests for {@link KeyedPValueTrackingVisitor}. */ @RunWith(JUnit4.class) public class KeyedPValueTrackingVisitorTest { @Rule public ExpectedException thrown = ExpectedException.none(); @@ -61,8 +65,7 @@ public void setup() { @Test public void groupByKeyProducesKeyedOutput() { PCollection>> keyed = - p.apply(Create.of(KV.of("foo", 3))) - .apply(GroupByKey.create()); + p.apply(Create.of(KV.of("foo", 3))).apply(GroupByKey.create()); p.traverseTopologically(visitor); assertThat(visitor.getKeyedPValues(), hasItem(keyed)); @@ -90,17 +93,67 @@ public void keyedInputWithoutKeyPreserving() { assertThat(visitor.getKeyedPValues(), not(hasItem(onceKeyed))); } + @Test + public void unkeyedInputWithKeyPreserving() { + + PCollection>>>> input = + p.apply( + Create.of( + KV.of( + "hello", + (Iterable>>) + Collections.>>emptyList())) + .withCoder( + KvCoder.of( + StringUtf8Coder.of(), + IterableCoder.of( + WindowedValue.getValueOnlyCoder( + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())))))); + + PCollection>> unkeyed = + input.apply(ParDo.of(new ParDoMultiOverrideFactory.ToKeyedWorkItem())); + + p.traverseTopologically(visitor); + assertThat(visitor.getKeyedPValues(), not(hasItem(unkeyed))); + } + + @Test + public void keyedInputWithKeyPreserving() { + + PCollection>>> input = + p.apply( + Create.of( + KV.of( + "hello", + WindowedValue.of( + KV.of("hello", 3), + new Instant(0), + new IntervalWindow(new Instant(0), new Instant(9)), + PaneInfo.NO_FIRING))) + .withCoder( + KvCoder.of( + StringUtf8Coder.of(), + WindowedValue.getValueOnlyCoder( + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))))); + + PCollection>> keyed = + input + .apply(GroupByKey.>>create()) + .apply(ParDo.of(new ParDoMultiOverrideFactory.ToKeyedWorkItem())); + + p.traverseTopologically(visitor); + assertThat(visitor.getKeyedPValues(), hasItem(keyed)); + } + @Test public void traverseMultipleTimesThrows() { p.apply( - Create.>of( - KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null)) + Create.of(KV.of(1, (Void) null), KV.of(2, (Void) null), KV.of(3, (Void) null)) .withCoder(KvCoder.of(VarIntCoder.of(), VoidCoder.of()))) .apply(GroupByKey.create()) .apply(Keys.create()); p.traverseTopologically(visitor); - thrown.expect(IllegalStateException.class); thrown.expectMessage("already been finalized"); thrown.expectMessage(KeyedPValueTrackingVisitor.class.getSimpleName()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index d312aa36f770..b88d5e0f6e66 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -27,12 +27,14 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.beam.runners.core.KeyedWorkItem; +import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; @@ -136,7 +138,7 @@ public void process(ProcessContext c) {} new StatefulParDoEvaluatorFactory(mockEvaluationContext); AppliedPTransform< - PCollection>>, PCollectionTuple, + PCollection>>, PCollectionTuple, StatefulParDo> producingTransform = (AppliedPTransform) DirectGraphs.getProducer(produced); @@ -245,7 +247,7 @@ public void process(ProcessContext c) {} // This will be the stateful ParDo from the expansion AppliedPTransform< - PCollection>>, PCollectionTuple, + PCollection>>, PCollectionTuple, StatefulParDo> producingTransform = (AppliedPTransform) DirectGraphs.getProducer(produced); @@ -270,37 +272,50 @@ public void process(ProcessContext c) {} // A single bundle with some elements in the global window; it should register cleanup for the // global window state merely by having the evaluator created. The cleanup logic does not // depend on the window. - WindowedValue>> gbkOutputElement = - WindowedValue.of( - KV.>of("hello", Lists.newArrayList(1, 13, 15)), - new Instant(3), - firstWindow, - PaneInfo.NO_FIRING); - CommittedBundle>> inputBundle = + String key = "hello"; + WindowedValue> firstKv = WindowedValue.of( + KV.of(key, 1), + new Instant(3), + firstWindow, + PaneInfo.NO_FIRING); + + WindowedValue>> gbkOutputElement = + firstKv.withValue( + KeyedWorkItems.elementsWorkItem( + "hello", + ImmutableList.of( + firstKv, + firstKv.withValue(KV.of(key, 13)), + firstKv.withValue(KV.of(key, 15))))); + + CommittedBundle>> inputBundle = BUNDLE_FACTORY .createBundle(producingTransform.getInput()) .add(gbkOutputElement) .commit(Instant.now()); - TransformEvaluator>> evaluator = + TransformEvaluator>> evaluator = factory.forApplication(producingTransform, inputBundle); + evaluator.processElement(gbkOutputElement); // This should push back every element as a KV> // in the appropriate window. Since the keys are equal they are single-threaded - TransformResult>> result = evaluator.finishBundle(); + TransformResult>> result = + evaluator.finishBundle(); List pushedBackInts = new ArrayList<>(); - for (WindowedValue unprocessedElement : result.getUnprocessedElements()) { - WindowedValue>> unprocessedKv = - (WindowedValue>>) unprocessedElement; + for (WindowedValue>> unprocessedElement : + result.getUnprocessedElements()) { assertThat( Iterables.getOnlyElement(unprocessedElement.getWindows()), equalTo((BoundedWindow) firstWindow)); - assertThat(unprocessedKv.getValue().getKey(), equalTo("hello")); - for (Integer i : unprocessedKv.getValue().getValue()) { - pushedBackInts.add(i); + + assertThat(unprocessedElement.getValue().key(), equalTo("hello")); + for (WindowedValue> windowedKv : + unprocessedElement.getValue().elementsIterable()) { + pushedBackInts.add(windowedKv.getValue().getValue()); } } assertThat(pushedBackInts, containsInAnyOrder(1, 13, 15)); From 445c120510948fb23e6d35b502da1e5a4f0ffdfb Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 20:45:56 -0800 Subject: [PATCH 260/279] Move InMemoryTimerInternals to runners-core --- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 1 - .../runners/core}/InMemoryTimerInternals.java | 8 ++--- .../core}/InMemoryTimerInternalsTest.java | 4 ++- .../beam/runners/core/ReduceFnTester.java | 1 - .../runners/core/SplittableParDoTest.java | 16 +++++++-- .../triggers/TriggerStateMachineTester.java | 2 +- .../translation/SparkGroupAlsoByWindowFn.java | 2 +- .../beam/sdk/transforms/DoFnTester.java | 36 ------------------- 8 files changed, 21 insertions(+), 49 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util/state => runners/core-java/src/main/java/org/apache/beam/runners/core}/InMemoryTimerInternals.java (97%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util/state => runners/core-java/src/test/java/org/apache/beam/runners/core}/InMemoryTimerInternalsTest.java (97%) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index 918919170ec6..efcd771d01b9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index 44b44f06ead6..5fcd088c9400 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -29,12 +29,10 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowTracing; +import org.apache.beam.sdk.util.state.StateNamespace; import org.joda.time.Instant; -/** - * Simulates the firing of timers and progression of input and output watermarks for a single - * computation and key in a Windmill-like streaming environment. - */ +/** {@link TimerInternals} with all watermarks and processing clock simulated in-memory. */ public class InMemoryTimerInternals implements TimerInternals { /** At most one timer per timestamp is kept. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java similarity index 97% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java index 4a2763ccc76e..2caa8742dea1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.runners.core; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -25,6 +25,8 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaceForTest; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index db0cf9186a7d..890195a0d3c1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -73,7 +73,6 @@ import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index 0f0b106d2ebd..74a566b35613 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -28,6 +28,7 @@ import static org.junit.Assert.assertTrue; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -196,6 +197,8 @@ private static class ProcessFnTester< tester; private Instant currentProcessingTime; + private InMemoryTimerInternals timerInternals; + ProcessFnTester( Instant currentProcessingTime, DoFn fn, @@ -206,6 +209,7 @@ private static class ProcessFnTester< new SplittableParDo.ProcessFn<>( fn, inputCoder, restrictionCoder, IntervalWindow.getCoder()); this.tester = DoFnTester.of(processFn); + this.timerInternals = new InMemoryTimerInternals(); processFn.setStateInternalsFactory( new StateInternalsFactory() { @Override @@ -217,7 +221,7 @@ public StateInternals stateInternalsForKey(String key) { new TimerInternalsFactory() { @Override public TimerInternals timerInternalsForKey(String key) { - return tester.getTimerInternals(); + return timerInternals; } }); processFn.setOutputWindowedValue( @@ -253,7 +257,7 @@ public void sideOutputWindowedValue( // through the state/timer/output callbacks. this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE); this.tester.startBundle(); - this.tester.advanceProcessingTime(currentProcessingTime); + timerInternals.advanceProcessingTime(currentProcessingTime); this.currentProcessingTime = currentProcessingTime; } @@ -291,7 +295,13 @@ void startElement( */ boolean advanceProcessingTimeBy(Duration duration) throws Exception { currentProcessingTime = currentProcessingTime.plus(duration); - List timers = tester.advanceProcessingTime(currentProcessingTime); + timerInternals.advanceProcessingTime(currentProcessingTime); + + List timers = new ArrayList<>(); + TimerInternals.TimerData nextTimer; + while ((nextTimer = timerInternals.removeNextProcessingTimer()) != null) { + timers.add(nextTimer); + } if (timers.isEmpty()) { return false; } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index be63c0644e27..2a626d4311d5 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -34,6 +34,7 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.ActiveWindowSet; import org.apache.beam.runners.core.ActiveWindowSet.MergeCallback; +import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.MergingActiveWindowSet; import org.apache.beam.runners.core.NonMergingActiveWindowSet; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -46,7 +47,6 @@ import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java index 87d3f5059039..5432d58994af 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowFn.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.List; import org.apache.beam.runners.core.GroupAlsoByWindowsDoFn; +import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; import org.apache.beam.runners.core.SystemReduceFn; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 93b3f5954898..2d8684aaaff7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -46,12 +46,10 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.util.state.InMemoryStateInternals; -import org.apache.beam.sdk.util.state.InMemoryTimerInternals; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; @@ -143,10 +141,6 @@ public StateInternals getStateInternals() { return (StateInternals) stateInternals; } - public TimerInternals getTimerInternals() { - return timerInternals; - } - /** * When a {@link DoFnTester} should clone the {@link DoFn} under test and how it should manage * the lifecycle of the {@link DoFn}. @@ -233,7 +227,6 @@ public void startBundle() throws Exception { context.setupDelegateAggregators(); // State and timer internals are per-bundle. stateInternals = InMemoryStateInternals.forKey(new Object()); - timerInternals = new InMemoryTimerInternals(); try { fnInvoker.invokeStartBundle(context); } catch (UserCodeException e) { @@ -542,34 +535,6 @@ public AggregateT getAggregatorValue(Aggregator agg) return extractAggregatorValue(agg.getName(), agg.getCombineFn()); } - public List advanceInputWatermark(Instant newWatermark) { - try { - timerInternals.advanceInputWatermark(newWatermark); - final List firedTimers = new ArrayList<>(); - TimerInternals.TimerData timer; - while ((timer = timerInternals.removeNextEventTimer()) != null) { - firedTimers.add(timer); - } - return firedTimers; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public List advanceProcessingTime(Instant newProcessingTime) { - try { - timerInternals.advanceProcessingTime(newProcessingTime); - final List firedTimers = new ArrayList<>(); - TimerInternals.TimerData timer; - while ((timer = timerInternals.removeNextProcessingTimer()) != null) { - firedTimers.add(timer); - } - return firedTimers; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - private AggregateT extractAggregatorValue( String name, CombineFn combiner) { @SuppressWarnings("unchecked") @@ -814,7 +779,6 @@ private enum State { private Map, List>> outputs; private InMemoryStateInternals stateInternals; - private InMemoryTimerInternals timerInternals; /** The state of processing of the {@link DoFn} under test. */ private State state = State.UNINITIALIZED; From 69d2c47b6a476099535e9cefe62d4cce5ccafbc1 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 16 Dec 2016 20:22:59 -0800 Subject: [PATCH 261/279] Restore SDK's InMemoryTimerInternals, deprecated --- .../util/state/InMemoryTimerInternals.java | 275 ++++++++++++++++++ 1 file changed, 275 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java new file mode 100644 index 000000000000..a910d647760f --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util.state; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.MoreObjects; +import java.util.HashSet; +import java.util.PriorityQueue; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowTracing; +import org.joda.time.Instant; + +/** + * @deprecated use {@code org.apache.beam.runners.core.InMemoryTimerInternals}. + */ +@Deprecated +public class InMemoryTimerInternals implements TimerInternals { + + /** At most one timer per timestamp is kept. */ + private Set existingTimers = new HashSet<>(); + + /** Pending input watermark timers, in timestamp order. */ + private PriorityQueue watermarkTimers = new PriorityQueue<>(11); + + /** Pending processing time timers, in timestamp order. */ + private PriorityQueue processingTimers = new PriorityQueue<>(11); + + /** Pending synchronized processing time timers, in timestamp order. */ + private PriorityQueue synchronizedProcessingTimers = new PriorityQueue<>(11); + + /** Current input watermark. */ + private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + + /** Current output watermark. */ + @Nullable private Instant outputWatermarkTime = null; + + /** Current processing time. */ + private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + + /** Current synchronized processing time. */ + private Instant synchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + + @Override + @Nullable + public Instant currentOutputWatermarkTime() { + return outputWatermarkTime; + } + + /** + * Returns when the next timer in the given time domain will fire, or {@code null} + * if there are no timers scheduled in that time domain. + */ + @Nullable + public Instant getNextTimer(TimeDomain domain) { + final TimerData data; + switch (domain) { + case EVENT_TIME: + data = watermarkTimers.peek(); + break; + case PROCESSING_TIME: + data = processingTimers.peek(); + break; + case SYNCHRONIZED_PROCESSING_TIME: + data = synchronizedProcessingTimers.peek(); + break; + default: + throw new IllegalArgumentException("Unexpected time domain: " + domain); + } + return (data == null) ? null : data.getTimestamp(); + } + + private PriorityQueue queue(TimeDomain domain) { + switch (domain) { + case EVENT_TIME: + return watermarkTimers; + case PROCESSING_TIME: + return processingTimers; + case SYNCHRONIZED_PROCESSING_TIME: + return synchronizedProcessingTimers; + default: + throw new IllegalArgumentException("Unexpected time domain: " + domain); + } + } + + @Override + public void setTimer(StateNamespace namespace, String timerId, Instant target, + TimeDomain timeDomain) { + throw new UnsupportedOperationException("Setting a timer by ID is not yet supported."); + } + + @Override + public void setTimer(TimerData timerData) { + WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData); + if (existingTimers.add(timerData)) { + queue(timerData.getDomain()).add(timerData); + } + } + + @Override + public void deleteTimer(StateNamespace namespace, String timerId) { + throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported."); + } + + @Override + public void deleteTimer(TimerData timer) { + WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer); + existingTimers.remove(timer); + queue(timer.getDomain()).remove(timer); + } + + @Override + public Instant currentProcessingTime() { + return processingTime; + } + + @Override + @Nullable + public Instant currentSynchronizedProcessingTime() { + return synchronizedProcessingTime; + } + + @Override + public Instant currentInputWatermarkTime() { + return inputWatermarkTime; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(getClass()) + .add("watermarkTimers", watermarkTimers) + .add("processingTimers", processingTimers) + .add("synchronizedProcessingTimers", synchronizedProcessingTimers) + .add("inputWatermarkTime", inputWatermarkTime) + .add("outputWatermarkTime", outputWatermarkTime) + .add("processingTime", processingTime) + .toString(); + } + + /** Advances input watermark to the given value. */ + public void advanceInputWatermark(Instant newInputWatermark) throws Exception { + checkNotNull(newInputWatermark); + checkState( + !newInputWatermark.isBefore(inputWatermarkTime), + "Cannot move input watermark time backwards from %s to %s", + inputWatermarkTime, + newInputWatermark); + WindowTracing.trace( + "{}.advanceInputWatermark: from {} to {}", + getClass().getSimpleName(), inputWatermarkTime, newInputWatermark); + inputWatermarkTime = newInputWatermark; + } + + /** Advances output watermark to the given value. */ + public void advanceOutputWatermark(Instant newOutputWatermark) { + checkNotNull(newOutputWatermark); + final Instant adjustedOutputWatermark; + if (newOutputWatermark.isAfter(inputWatermarkTime)) { + WindowTracing.trace( + "{}.advanceOutputWatermark: clipping output watermark from {} to {}", + getClass().getSimpleName(), newOutputWatermark, inputWatermarkTime); + adjustedOutputWatermark = inputWatermarkTime; + } else { + adjustedOutputWatermark = newOutputWatermark; + } + + checkState( + outputWatermarkTime == null || !adjustedOutputWatermark.isBefore(outputWatermarkTime), + "Cannot move output watermark time backwards from %s to %s", + outputWatermarkTime, + adjustedOutputWatermark); + WindowTracing.trace( + "{}.advanceOutputWatermark: from {} to {}", + getClass().getSimpleName(), outputWatermarkTime, adjustedOutputWatermark); + outputWatermarkTime = adjustedOutputWatermark; + } + + /** Advances processing time to the given value. */ + public void advanceProcessingTime(Instant newProcessingTime) throws Exception { + checkNotNull(newProcessingTime); + checkState( + !newProcessingTime.isBefore(processingTime), + "Cannot move processing time backwards from %s to %s", + processingTime, + newProcessingTime); + WindowTracing.trace( + "{}.advanceProcessingTime: from {} to {}", + getClass().getSimpleName(), processingTime, newProcessingTime); + processingTime = newProcessingTime; + } + + /** Advances synchronized processing time to the given value. */ + public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime) + throws Exception { + checkNotNull(newSynchronizedProcessingTime); + checkState( + !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime), + "Cannot move processing time backwards from %s to %s", + synchronizedProcessingTime, + newSynchronizedProcessingTime); + WindowTracing.trace( + "{}.advanceProcessingTime: from {} to {}", + getClass().getSimpleName(), synchronizedProcessingTime, newSynchronizedProcessingTime); + synchronizedProcessingTime = newSynchronizedProcessingTime; + } + + /** Returns the next eligible event time timer, if none returns null. */ + @Nullable + public TimerData removeNextEventTimer() { + TimerData timer = removeNextTimer(inputWatermarkTime, TimeDomain.EVENT_TIME); + if (timer != null) { + WindowTracing.trace( + "{}.removeNextEventTimer: firing {} at {}", + getClass().getSimpleName(), timer, inputWatermarkTime); + } + return timer; + } + + /** Returns the next eligible processing time timer, if none returns null. */ + @Nullable + public TimerData removeNextProcessingTimer() { + TimerData timer = removeNextTimer(processingTime, TimeDomain.PROCESSING_TIME); + if (timer != null) { + WindowTracing.trace( + "{}.removeNextProcessingTimer: firing {} at {}", + getClass().getSimpleName(), timer, processingTime); + } + return timer; + } + + /** Returns the next eligible synchronized processing time timer, if none returns null. */ + @Nullable + public TimerData removeNextSynchronizedProcessingTimer() { + TimerData timer = removeNextTimer( + synchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); + if (timer != null) { + WindowTracing.trace( + "{}.removeNextSynchronizedProcessingTimer: firing {} at {}", + getClass().getSimpleName(), timer, synchronizedProcessingTime); + } + return timer; + } + + @Nullable + private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) { + PriorityQueue queue = queue(domain); + if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { + TimerData timer = queue.remove(); + existingTimers.remove(timer); + return timer; + } else { + return null; + } + } +} From ef1a858347e475cd15f7dcd8873464f506527b2a Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Tue, 6 Dec 2016 14:19:12 -0800 Subject: [PATCH 262/279] Provide a better error message for non-existing gcpTempLocation gcpTempLocation will default to using the value for tmpLocation, as long as the value is a valid GCP path. Non-valid GCP paths are silently discarded. This change removes existence validation from the default value logic such that downstream validation can provide a better error message. --- .../beam/runners/dataflow/DataflowRunner.java | 25 +++++++---- .../options/DataflowPipelineOptions.java | 19 +++++---- .../runners/dataflow/DataflowRunnerTest.java | 42 ++++++++++++++----- .../options/DataflowPipelineOptionsTest.java | 20 +++++---- .../apache/beam/sdk/options/GcpOptions.java | 19 +++++---- .../beam/sdk/util/GcsPathValidator.java | 3 +- .../beam/sdk/options/GcpOptionsTest.java | 32 ++++++++++++-- .../beam/sdk/util/GcsPathValidatorTest.java | 15 +------ 8 files changed, 114 insertions(+), 61 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 711b1b0d0555..1a15eaf23205 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -244,14 +244,23 @@ public static DataflowRunner fromOptions(PipelineOptions options) { } PathValidator validator = dataflowOptions.getPathValidator(); - checkArgument( - !isNullOrEmpty(dataflowOptions.getGcpTempLocation()), - "DataflowRunner requires gcpTempLocation, and it is missing in PipelineOptions."); - validator.validateOutputFilePrefixSupported(dataflowOptions.getGcpTempLocation()); - checkArgument( - !isNullOrEmpty(dataflowOptions.getStagingLocation()), - "DataflowRunner requires stagingLocation, and it is missing in PipelineOptions."); - validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation()); + String gcpTempLocation; + try { + gcpTempLocation = dataflowOptions.getGcpTempLocation(); + } catch (Exception e) { + throw new IllegalArgumentException("DataflowRunner requires gcpTempLocation, " + + "but failed to retrieve a value from PipelineOptions", e); + } + validator.validateOutputFilePrefixSupported(gcpTempLocation); + + String stagingLocation; + try { + stagingLocation = dataflowOptions.getStagingLocation(); + } catch (Exception e) { + throw new IllegalArgumentException("DataflowRunner requires stagingLocation, " + + "but failed to retrieve a value from PipelineOptions", e); + } + validator.validateOutputFilePrefixSupported(stagingLocation); if (!Strings.isNullOrEmpty(dataflowOptions.getSaveProfilesToGcs())) { validator.validateOutputFilePrefixSupported(dataflowOptions.getSaveProfilesToGcs()); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java index 66632ad78a3e..5ddc5d0e33b2 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java @@ -17,9 +17,6 @@ */ package org.apache.beam.runners.dataflow.options; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.isNullOrEmpty; - import java.io.IOException; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.annotations.Experimental; @@ -107,17 +104,21 @@ class StagingLocationFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { GcsOptions gcsOptions = options.as(GcsOptions.class); - String gcpTempLocation = gcsOptions.getGcpTempLocation(); - checkArgument(!isNullOrEmpty(gcpTempLocation), - "Error constructing default value for stagingLocation: gcpTempLocation is missing." - + "Either stagingLocation must be set explicitly or a valid value must be provided" - + "for gcpTempLocation."); + String gcpTempLocation; + try { + gcpTempLocation = gcsOptions.getGcpTempLocation(); + } catch (Exception e) { + throw new IllegalArgumentException( + "Error constructing default value for stagingLocation: failed to retrieve gcpTempLocation. " + + "Either stagingLocation must be set explicitly or a valid value must be provided" + + "for gcpTempLocation.", e); + } try { gcsOptions.getPathValidator().validateOutputFilePrefixSupported(gcpTempLocation); } catch (Exception e) { throw new IllegalArgumentException(String.format( "Error constructing default value for stagingLocation: gcpTempLocation is not" - + " a valid GCS path, %s. ", gcpTempLocation)); + + " a valid GCS path, %s. ", gcpTempLocation), e); } try { return IOChannelUtils.resolve(gcpTempLocation, "staging"); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 4159b611e0b4..b29c4cdd6d26 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow; import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -279,6 +280,26 @@ public void testPathValidation() { } } + @Test + public void testPathExistsValidation() { + String[] args = new String[] { + "--runner=DataflowRunner", + "--tempLocation=gs://does/not/exist", + "--project=test-project", + "--credentialFactoryClass=" + NoopCredentialFactory.class.getCanonicalName(), + }; + + try { + TestPipeline.fromOptions(PipelineOptionsFactory.fromArgs(args).create()); + fail(); + } catch (RuntimeException e) { + assertThat( + Throwables.getStackTraceAsString(e), + both(containsString("gs://does/not/exist")) + .and(containsString("does not exist or is not writeable"))); + } + } + @Test public void testPathValidatorOverride() { String[] args = new String[] { @@ -544,7 +565,7 @@ public void testNonGcsFilePathInReadFailure() throws IOException { thrown.expectCause(Matchers.allOf( instanceOf(IllegalArgumentException.class), ThrowableMessageMatcher.hasMessage( - containsString("expected a valid 'gs://' path but was given")))); + containsString("Expected a valid 'gs://' path but was given")))); p.run(); ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); @@ -559,7 +580,7 @@ public void testNonGcsFilePathInWriteFailure() throws IOException { PCollection pc = p.apply("ReadMyGcsFile", TextIO.Read.from("gs://bucket/object")); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(containsString("expected a valid 'gs://' path but was given")); + thrown.expectMessage(containsString("Expected a valid 'gs://' path but was given")); pc.apply("WriteMyNonGcsFile", TextIO.Write.to("/tmp/file")); } @@ -594,7 +615,7 @@ public void testInvalidGcpTempLocation() throws IOException { options.setGcpTempLocation("file://temp/location"); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(containsString("expected a valid 'gs://' path but was given")); + thrown.expectMessage(containsString("Expected a valid 'gs://' path but was given")); DataflowRunner.fromOptions(options); ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); @@ -609,7 +630,8 @@ public void testNonGcsTempLocation() throws IOException { thrown.expect(IllegalArgumentException.class); thrown.expectMessage( - "DataflowRunner requires gcpTempLocation, and it is missing in PipelineOptions."); + "DataflowRunner requires gcpTempLocation, " + + "but failed to retrieve a value from PipelineOptions"); DataflowRunner.fromOptions(options); } @@ -621,14 +643,14 @@ public void testInvalidStagingLocation() throws IOException { DataflowRunner.fromOptions(options); fail("fromOptions should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given")); + assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given")); } options.setStagingLocation("my/staging/location"); try { DataflowRunner.fromOptions(options); fail("fromOptions should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given")); + assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given")); } } @@ -640,14 +662,14 @@ public void testInvalidProfileLocation() throws IOException { DataflowRunner.fromOptions(options); fail("fromOptions should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given")); + assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given")); } options.setSaveProfilesToGcs("my/staging/location"); try { DataflowRunner.fromOptions(options); fail("fromOptions should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given")); + assertThat(e.getMessage(), containsString("Expected a valid 'gs://' path but was given")); } } @@ -795,8 +817,8 @@ public void testNoStagingLocationAndNoTempLocationFails() { options.setProject("foo-project"); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "DataflowRunner requires gcpTempLocation, and it is missing in PipelineOptions."); + thrown.expectMessage("DataflowRunner requires gcpTempLocation, " + + "but failed to retrieve a value from PipelineOption"); DataflowRunner.fromOptions(options); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java index 52082e08865c..9dacfb2c5b24 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java @@ -17,9 +17,10 @@ */ package org.apache.beam.runners.dataflow.options; -import static com.google.common.base.Strings.isNullOrEmpty; +import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ResetDateTimeProvider; @@ -127,9 +128,10 @@ public void testUtf8UserNameAndApplicationNameIsNormalized() { public void testStagingLocation() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); IOChannelUtils.registerIOFactoriesAllowOverride(options); - options.setTempLocation("file://temp_location"); + options.setPathValidatorClass(NoopPathValidator.class); + options.setTempLocation("gs://temp_location"); options.setStagingLocation("gs://staging_location"); - assertTrue(isNullOrEmpty(options.getGcpTempLocation())); + assertEquals("gs://temp_location", options.getGcpTempLocation()); assertEquals("gs://staging_location", options.getStagingLocation()); } @@ -158,8 +160,10 @@ public void testDefaultNoneGcsTempLocation() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setTempLocation("file://temp_location"); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "Error constructing default value for stagingLocation: gcpTempLocation is missing."); + thrown.expectMessage("Error constructing default value for stagingLocation: " + + "failed to retrieve gcpTempLocation."); + thrown.expectCause(hasMessage(containsString( + "Error constructing default value for gcpTempLocation"))); options.getStagingLocation(); } @@ -171,6 +175,8 @@ public void testDefaultInvalidGcpTempLocation() { thrown.expectMessage( "Error constructing default value for stagingLocation: gcpTempLocation is not" + " a valid GCS path"); + thrown.expectCause( + hasMessage(containsString("Expected a valid 'gs://' path"))); options.getStagingLocation(); } @@ -178,8 +184,8 @@ public void testDefaultInvalidGcpTempLocation() { public void testDefaultStagingLocationUnset() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "Error constructing default value for stagingLocation: gcpTempLocation is missing."); + thrown.expectMessage("Error constructing default value for stagingLocation: " + + "failed to retrieve gcpTempLocation."); options.getStagingLocation(); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java index ffdab9838e05..042f4b483f29 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcpOptions.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.options; +import static com.google.common.base.Preconditions.checkArgument; + import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.auth.Credentials; import com.google.common.annotations.VisibleForTesting; @@ -195,14 +197,15 @@ class GcpTempLocationFactory implements DefaultValueFactory { @Nullable public String create(PipelineOptions options) { String tempLocation = options.getTempLocation(); - if (!Strings.isNullOrEmpty(tempLocation)) { - try { - PathValidator validator = options.as(GcsOptions.class).getPathValidator(); - validator.validateOutputFilePrefixSupported(tempLocation); - } catch (Exception e) { - // Ignore the temp location because it is not a valid 'gs://' path. - return null; - } + checkArgument(!Strings.isNullOrEmpty(options.getTempLocation()), + "Error constructing default value for gcpTempLocation: tempLocation is not set"); + try { + PathValidator validator = options.as(GcsOptions.class).getPathValidator(); + validator.validateOutputFilePrefixSupported(tempLocation); + } catch (Exception e) { + throw new IllegalArgumentException(String.format( + "Error constructing default value for gcpTempLocation: tempLocation is not" + + " a valid GCS path, %s. ", tempLocation), e); } return tempLocation; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java index c8da4d8a8fef..a5b951de1aaa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java @@ -89,8 +89,7 @@ private GcsPath getGcsPath(String path) { return GcsPath.fromUri(path); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(String.format( - "%s expected a valid 'gs://' path but was given '%s'", - gcpOptions.getRunner().getSimpleName(), path), e); + "Expected a valid 'gs://' path but was given '%s'", path), e); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java index 34077a2bdfcd..7854d67768be 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GcpOptionsTest.java @@ -17,10 +17,10 @@ */ package org.apache.beam.sdk.options; -import static com.google.common.base.Strings.isNullOrEmpty; +import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -35,6 +35,7 @@ import org.apache.beam.sdk.util.NoopPathValidator; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestRule; import org.junit.runner.RunWith; @@ -45,6 +46,7 @@ public class GcpOptionsTest { @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule public ExpectedException thrown = ExpectedException.none(); @Test public void testGetProjectFromCloudSdkConfigEnv() throws Exception { @@ -107,7 +109,10 @@ public void testUnableToGetDefaultProject() throws Exception { @Test public void testEmptyGcpTempLocation() throws Exception { GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class); - assertTrue(isNullOrEmpty(options.getGcpTempLocation())); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Error constructing default value for gcpTempLocation: tempLocation is not set"); + options.getGcpTempLocation(); } @Test @@ -123,7 +128,26 @@ public void testDefaultGcpTempLocation() throws Exception { public void testDefaultGcpTempLocationInvalid() throws Exception { GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class); options.setTempLocation("file://"); - assertTrue(isNullOrEmpty(options.getGcpTempLocation())); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Error constructing default value for gcpTempLocation: tempLocation is not" + + " a valid GCS path"); + options.getGcpTempLocation(); + } + + @Test + public void testDefaultGcpTempLocationDoesNotExist() { + GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class); + String tempLocation = "gs://does/not/exist"; + options.setTempLocation(tempLocation); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Error constructing default value for gcpTempLocation: tempLocation is not" + + " a valid GCS path"); + thrown.expectCause( + hasMessage(containsString("Output path does not exist or is not writeable"))); + + options.getGcpTempLocation(); } private static void makePropertiesFileWithProject(File path, String projectId) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java index 16f01a0367f8..286490dc1462 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsPathValidatorTest.java @@ -21,11 +21,8 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.junit.Before; import org.junit.Rule; @@ -44,20 +41,12 @@ public class GcsPathValidatorTest { @Mock private GcsUtil mockGcsUtil; private GcsPathValidator validator; - private class FakeRunner extends PipelineRunner { - @Override - public PipelineResult run(Pipeline pipeline) { - throw new UnsupportedOperationException(); - } - } - @Before public void setUp() throws Exception { MockitoAnnotations.initMocks(this); when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(true); when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod(); GcsOptions options = PipelineOptionsFactory.as(GcsOptions.class); - options.setRunner(FakeRunner.class); options.setGcpCredential(new TestCredential()); options.setGcsUtil(mockGcsUtil); validator = GcsPathValidator.fromOptions(options); @@ -72,7 +61,7 @@ public void testValidFilePattern() { public void testInvalidFilePattern() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage( - "FakeRunner expected a valid 'gs://' path but was given '/local/path'"); + "Expected a valid 'gs://' path but was given '/local/path'"); validator.validateInputFilePatternSupported("/local/path"); } @@ -94,7 +83,7 @@ public void testValidOutputPrefix() { public void testInvalidOutputPrefix() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage( - "FakeRunner expected a valid 'gs://' path but was given '/local/path'"); + "Expected a valid 'gs://' path but was given '/local/path'"); validator.validateOutputFilePrefixSupported("/local/path"); } } From 96d393147c365d0911a091d0b3600fef621709f4 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Tue, 20 Dec 2016 11:47:42 -0800 Subject: [PATCH 263/279] Fixup usage of canonical name with name since canonical name != name for inner classes. --- .../apache/beam/runners/dataflow/DataflowRunnerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index b29c4cdd6d26..21d575aeb798 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -267,7 +267,7 @@ public void testPathValidation() { "--runner=DataflowRunner", "--tempLocation=/tmp/not/a/gs/path", "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getCanonicalName(), + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), }; try { @@ -286,7 +286,7 @@ public void testPathExistsValidation() { "--runner=DataflowRunner", "--tempLocation=gs://does/not/exist", "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getCanonicalName(), + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), }; try { @@ -306,8 +306,8 @@ public void testPathValidatorOverride() { "--runner=DataflowRunner", "--tempLocation=/tmp/testing", "--project=test-project", - "--credentialFactoryClass=" + NoopCredentialFactory.class.getCanonicalName(), - "--pathValidatorClass=" + NoopPathValidator.class.getCanonicalName(), + "--credentialFactoryClass=" + NoopCredentialFactory.class.getName(), + "--pathValidatorClass=" + NoopPathValidator.class.getName(), }; // Should not crash, because gcpTempLocation should get set from tempLocation TestPipeline.fromOptions(PipelineOptionsFactory.fromArgs(args).create()); From a3f68d343d018d99a04ac5c9134a11f1bc74935b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 14:05:19 -0800 Subject: [PATCH 264/279] Update Dataflow worker to beam-master-20161220 --- .../org/apache/beam/runners/dataflow/dataflow.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index 27a518f02713..bf08e8394c3c 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -18,6 +18,6 @@ environment.major.version=6 -worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161216 +worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161220 -worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161216 +worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161220 From aab46a0ec6e0e45208f64de7aabb9af643acd0ec Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 20:13:25 -0800 Subject: [PATCH 265/279] Remove deprecated AggregatorFactory from SDK --- .../beam/sdk/transforms/Aggregator.java | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java index 43f53a8d5f94..4119c531c138 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Aggregator.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.transforms; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.ExecutionContext; /** * An {@code Aggregator} enables monitoring of values of type {@code InputT}, @@ -68,22 +67,4 @@ public interface Aggregator { * aggregator. */ CombineFn getCombineFn(); - - /** - * @deprecated this is for use only by runners and exists only for a migration period. Please - * use the identical interface in org.apache.beam.runners.core - */ - @Deprecated - interface AggregatorFactory { - /** - * Create an aggregator with the given {@code name} and {@link CombineFn}. - * - *

      This method is called to create an aggregator for a {@link DoFn}. It receives the - * class of the {@link DoFn} being executed and the context of the step it is being - * executed in. - */ - Aggregator createAggregatorForDoFn( - Class fnClass, ExecutionContext.StepContext stepContext, - String aggregatorName, CombineFn combine); - } } From 9f1d3d155303bd3d1069541be704d5f3e74926eb Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 14:07:00 -0800 Subject: [PATCH 266/279] Remove deprecated InMemoryTimerInternals from SDK --- .../util/state/InMemoryTimerInternals.java | 275 ------------------ 1 file changed, 275 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java deleted file mode 100644 index a910d647760f..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util.state; - -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - -import com.google.common.base.MoreObjects; -import java.util.HashSet; -import java.util.PriorityQueue; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.TimerInternals; -import org.apache.beam.sdk.util.WindowTracing; -import org.joda.time.Instant; - -/** - * @deprecated use {@code org.apache.beam.runners.core.InMemoryTimerInternals}. - */ -@Deprecated -public class InMemoryTimerInternals implements TimerInternals { - - /** At most one timer per timestamp is kept. */ - private Set existingTimers = new HashSet<>(); - - /** Pending input watermark timers, in timestamp order. */ - private PriorityQueue watermarkTimers = new PriorityQueue<>(11); - - /** Pending processing time timers, in timestamp order. */ - private PriorityQueue processingTimers = new PriorityQueue<>(11); - - /** Pending synchronized processing time timers, in timestamp order. */ - private PriorityQueue synchronizedProcessingTimers = new PriorityQueue<>(11); - - /** Current input watermark. */ - private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; - - /** Current output watermark. */ - @Nullable private Instant outputWatermarkTime = null; - - /** Current processing time. */ - private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; - - /** Current synchronized processing time. */ - private Instant synchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; - - @Override - @Nullable - public Instant currentOutputWatermarkTime() { - return outputWatermarkTime; - } - - /** - * Returns when the next timer in the given time domain will fire, or {@code null} - * if there are no timers scheduled in that time domain. - */ - @Nullable - public Instant getNextTimer(TimeDomain domain) { - final TimerData data; - switch (domain) { - case EVENT_TIME: - data = watermarkTimers.peek(); - break; - case PROCESSING_TIME: - data = processingTimers.peek(); - break; - case SYNCHRONIZED_PROCESSING_TIME: - data = synchronizedProcessingTimers.peek(); - break; - default: - throw new IllegalArgumentException("Unexpected time domain: " + domain); - } - return (data == null) ? null : data.getTimestamp(); - } - - private PriorityQueue queue(TimeDomain domain) { - switch (domain) { - case EVENT_TIME: - return watermarkTimers; - case PROCESSING_TIME: - return processingTimers; - case SYNCHRONIZED_PROCESSING_TIME: - return synchronizedProcessingTimers; - default: - throw new IllegalArgumentException("Unexpected time domain: " + domain); - } - } - - @Override - public void setTimer(StateNamespace namespace, String timerId, Instant target, - TimeDomain timeDomain) { - throw new UnsupportedOperationException("Setting a timer by ID is not yet supported."); - } - - @Override - public void setTimer(TimerData timerData) { - WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData); - if (existingTimers.add(timerData)) { - queue(timerData.getDomain()).add(timerData); - } - } - - @Override - public void deleteTimer(StateNamespace namespace, String timerId) { - throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported."); - } - - @Override - public void deleteTimer(TimerData timer) { - WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer); - existingTimers.remove(timer); - queue(timer.getDomain()).remove(timer); - } - - @Override - public Instant currentProcessingTime() { - return processingTime; - } - - @Override - @Nullable - public Instant currentSynchronizedProcessingTime() { - return synchronizedProcessingTime; - } - - @Override - public Instant currentInputWatermarkTime() { - return inputWatermarkTime; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("watermarkTimers", watermarkTimers) - .add("processingTimers", processingTimers) - .add("synchronizedProcessingTimers", synchronizedProcessingTimers) - .add("inputWatermarkTime", inputWatermarkTime) - .add("outputWatermarkTime", outputWatermarkTime) - .add("processingTime", processingTime) - .toString(); - } - - /** Advances input watermark to the given value. */ - public void advanceInputWatermark(Instant newInputWatermark) throws Exception { - checkNotNull(newInputWatermark); - checkState( - !newInputWatermark.isBefore(inputWatermarkTime), - "Cannot move input watermark time backwards from %s to %s", - inputWatermarkTime, - newInputWatermark); - WindowTracing.trace( - "{}.advanceInputWatermark: from {} to {}", - getClass().getSimpleName(), inputWatermarkTime, newInputWatermark); - inputWatermarkTime = newInputWatermark; - } - - /** Advances output watermark to the given value. */ - public void advanceOutputWatermark(Instant newOutputWatermark) { - checkNotNull(newOutputWatermark); - final Instant adjustedOutputWatermark; - if (newOutputWatermark.isAfter(inputWatermarkTime)) { - WindowTracing.trace( - "{}.advanceOutputWatermark: clipping output watermark from {} to {}", - getClass().getSimpleName(), newOutputWatermark, inputWatermarkTime); - adjustedOutputWatermark = inputWatermarkTime; - } else { - adjustedOutputWatermark = newOutputWatermark; - } - - checkState( - outputWatermarkTime == null || !adjustedOutputWatermark.isBefore(outputWatermarkTime), - "Cannot move output watermark time backwards from %s to %s", - outputWatermarkTime, - adjustedOutputWatermark); - WindowTracing.trace( - "{}.advanceOutputWatermark: from {} to {}", - getClass().getSimpleName(), outputWatermarkTime, adjustedOutputWatermark); - outputWatermarkTime = adjustedOutputWatermark; - } - - /** Advances processing time to the given value. */ - public void advanceProcessingTime(Instant newProcessingTime) throws Exception { - checkNotNull(newProcessingTime); - checkState( - !newProcessingTime.isBefore(processingTime), - "Cannot move processing time backwards from %s to %s", - processingTime, - newProcessingTime); - WindowTracing.trace( - "{}.advanceProcessingTime: from {} to {}", - getClass().getSimpleName(), processingTime, newProcessingTime); - processingTime = newProcessingTime; - } - - /** Advances synchronized processing time to the given value. */ - public void advanceSynchronizedProcessingTime(Instant newSynchronizedProcessingTime) - throws Exception { - checkNotNull(newSynchronizedProcessingTime); - checkState( - !newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime), - "Cannot move processing time backwards from %s to %s", - synchronizedProcessingTime, - newSynchronizedProcessingTime); - WindowTracing.trace( - "{}.advanceProcessingTime: from {} to {}", - getClass().getSimpleName(), synchronizedProcessingTime, newSynchronizedProcessingTime); - synchronizedProcessingTime = newSynchronizedProcessingTime; - } - - /** Returns the next eligible event time timer, if none returns null. */ - @Nullable - public TimerData removeNextEventTimer() { - TimerData timer = removeNextTimer(inputWatermarkTime, TimeDomain.EVENT_TIME); - if (timer != null) { - WindowTracing.trace( - "{}.removeNextEventTimer: firing {} at {}", - getClass().getSimpleName(), timer, inputWatermarkTime); - } - return timer; - } - - /** Returns the next eligible processing time timer, if none returns null. */ - @Nullable - public TimerData removeNextProcessingTimer() { - TimerData timer = removeNextTimer(processingTime, TimeDomain.PROCESSING_TIME); - if (timer != null) { - WindowTracing.trace( - "{}.removeNextProcessingTimer: firing {} at {}", - getClass().getSimpleName(), timer, processingTime); - } - return timer; - } - - /** Returns the next eligible synchronized processing time timer, if none returns null. */ - @Nullable - public TimerData removeNextSynchronizedProcessingTimer() { - TimerData timer = removeNextTimer( - synchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME); - if (timer != null) { - WindowTracing.trace( - "{}.removeNextSynchronizedProcessingTimer: firing {} at {}", - getClass().getSimpleName(), timer, synchronizedProcessingTime); - } - return timer; - } - - @Nullable - private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) { - PriorityQueue queue = queue(domain); - if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { - TimerData timer = queue.remove(); - existingTimers.remove(timer); - return timer; - } else { - return null; - } - } -} From 34373c21ed67696235d88ef40d50e31c77b84c33 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 6 Dec 2016 11:03:52 -0800 Subject: [PATCH 267/279] Provide local tags in PInput, POutput expansions Output an ordered colleciton in PInput and POutput expansions. This provides information that is necessary to reconstruct a PInput or POutput from its expansion. Implement PCollectionList.equals, PCollectionTuple.equals --- .../runners/direct/DirectGraphVisitor.java | 18 +-- .../runners/direct/EvaluationContext.java | 7 +- .../direct/KeyedPValueTrackingVisitor.java | 16 ++- .../beam/runners/direct/WatermarkManager.java | 19 +-- .../beam/runners/spark/SparkRunner.java | 13 +- .../beam/sdk/runners/TransformHierarchy.java | 49 ++++---- .../join/KeyedPCollectionTuple.java | 9 +- .../org/apache/beam/sdk/values/PBegin.java | 4 +- .../beam/sdk/values/PCollectionList.java | 65 +++++++--- .../beam/sdk/values/PCollectionTuple.java | 28 ++++- .../org/apache/beam/sdk/values/PDone.java | 4 +- .../org/apache/beam/sdk/values/PInput.java | 4 +- .../org/apache/beam/sdk/values/POutput.java | 4 +- .../org/apache/beam/sdk/values/PValue.java | 10 ++ .../apache/beam/sdk/values/PValueBase.java | 11 +- .../apache/beam/sdk/values/TaggedPValue.java | 42 +++++++ .../sdk/runners/TransformHierarchyTest.java | 23 +++- .../apache/beam/sdk/transforms/ParDoTest.java | 34 +++++ .../beam/sdk/values/PCollectionListTest.java | 117 ++++++++++++++++++ .../beam/sdk/values/PCollectionTupleTest.java | 70 +++++++++++ 20 files changed, 449 insertions(+), 98 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index 0283d0352240..425bbf10d927 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -35,6 +35,7 @@ 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.TaggedPValue; /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the @@ -79,14 +80,16 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { - toFinalize.removeAll(node.getInputs()); + for (TaggedPValue consumed : node.getInputs()) { + toFinalize.remove(consumed.getValue()); + } AppliedPTransform appliedTransform = getAppliedTransform(node); stepNames.put(appliedTransform, genStepName()); if (node.getInputs().isEmpty()) { rootTransforms.add(appliedTransform); } else { - for (PValue value : node.getInputs()) { - primitiveConsumers.put(value, appliedTransform); + for (TaggedPValue value : node.getInputs()) { + primitiveConsumers.put(value.getValue(), appliedTransform); } } } @@ -96,15 +99,12 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { toFinalize.add(value); AppliedPTransform appliedTransform = getAppliedTransform(producer); + if (value instanceof PCollectionView) { + views.add((PCollectionView) value); + } if (!producers.containsKey(value)) { producers.put(value, appliedTransform); } - if (value instanceof PCollectionView) { - views.add((PCollectionView) value); - } - if (!producers.containsKey(value)) { - producers.put(value, appliedTransform); - } } private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index cb9ddd899800..bbcab8e4b8a0 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -53,6 +53,7 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; import org.joda.time.Instant; /** @@ -419,9 +420,9 @@ public boolean isDone(AppliedPTransform transform) { } // If the PTransform has any unbounded outputs, and unbounded producers should not be shut down, // the PTransform may produce additional output. It is not done. - for (PValue output : transform.getOutput().expand()) { - if (output instanceof PCollection) { - IsBounded bounded = ((PCollection) output).isBounded(); + for (TaggedPValue output : transform.getOutput().expand()) { + if (output.getValue() instanceof PCollection) { + IsBounded bounded = ((PCollection) output.getValue()).isBounded(); if (bounded.equals(IsBounded.UNBOUNDED) && !options.isShutdownUnboundedProducersWithMaxWatermark()) { return false; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 65c41e06ba9f..32eb692ccbab 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -18,11 +18,10 @@ package org.apache.beam.runners.direct; import static com.google.common.base.Preconditions.checkState; -import static com.google.common.base.Predicates.in; -import static com.google.common.collect.Iterables.all; import com.google.common.collect.ImmutableSet; import java.util.HashSet; +import java.util.List; import java.util.Set; import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; @@ -33,6 +32,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; /** * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it @@ -83,7 +83,10 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { if (node.isRootNode()) { finalized = true; } else if (PRODUCES_KEYED_OUTPUTS.contains(node.getTransform().getClass())) { - keyedValues.addAll(node.getOutputs()); + List outputs = node.getOutputs(); + for (TaggedPValue output : outputs) { + keyedValues.add(output.getValue()); + } } } @@ -92,9 +95,12 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) {} @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { + boolean inputsAreKeyed = true; + for (TaggedPValue input : producer.getInputs()) { + inputsAreKeyed = inputsAreKeyed && keyedValues.contains(input.getValue()); + } if (PRODUCES_KEYED_OUTPUTS.contains(producer.getTransform().getClass()) - || (isKeyPreserving(producer.getTransform()) - && all(producer.getInputs(), in(keyedValues)))) { + || (isKeyPreserving(producer.getTransform()) && inputsAreKeyed)) { keyedValues.add(value); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index 247b1cce9ab4..7bed75107f4a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -57,7 +57,7 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; import org.joda.time.Instant; /** @@ -755,13 +755,14 @@ private TransformWatermarks getTransformWatermark(AppliedPTransform tra private Collection getInputProcessingWatermarks(AppliedPTransform transform) { ImmutableList.Builder inputWmsBuilder = ImmutableList.builder(); - Collection inputs = transform.getInput().expand(); + List inputs = transform.getInput().expand(); if (inputs.isEmpty()) { inputWmsBuilder.add(THE_END_OF_TIME); } - for (PValue pvalue : inputs) { + for (TaggedPValue pvalue : inputs) { Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).synchronizedProcessingOutputWatermark; + getTransformWatermark(graph.getProducer(pvalue.getValue())) + .synchronizedProcessingOutputWatermark; inputWmsBuilder.add(producerOutputWatermark); } return inputWmsBuilder.build(); @@ -769,13 +770,13 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) { ImmutableList.Builder inputWatermarksBuilder = ImmutableList.builder(); - Collection inputs = transform.getInput().expand(); + List inputs = transform.getInput().expand(); if (inputs.isEmpty()) { inputWatermarksBuilder.add(THE_END_OF_TIME); } - for (PValue pvalue : inputs) { + for (TaggedPValue pvalue : inputs) { Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; + getTransformWatermark(graph.getProducer(pvalue.getValue())).outputWatermark; inputWatermarksBuilder.add(producerOutputWatermark); } List inputCollectionWatermarks = inputWatermarksBuilder.build(); @@ -959,8 +960,8 @@ synchronized void refreshAll() { WatermarkUpdate updateResult = myWatermarks.refresh(); if (updateResult.isAdvanced()) { Set> additionalRefreshes = new HashSet<>(); - for (PValue outputPValue : toRefresh.getOutput().expand()) { - additionalRefreshes.addAll(graph.getPrimitiveConsumers(outputPValue)); + for (TaggedPValue outputPValue : toRefresh.getOutput().expand()) { + additionalRefreshes.addAll(graph.getPrimitiveConsumers(outputPValue.getValue())); } return additionalRefreshes; } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 3d98b87a351f..92c07bb4b12b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -48,6 +48,7 @@ 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.TaggedPValue; import org.apache.spark.Accumulator; import org.apache.spark.SparkEnv$; import org.apache.spark.api.java.JavaSparkContext; @@ -282,7 +283,7 @@ private boolean shouldDefer(TransformHierarchy.Node node) { if (node.getInputs().size() != 1) { return false; } - PValue input = Iterables.getOnlyElement(node.getInputs()); + PValue input = Iterables.getOnlyElement(node.getInputs()).getValue(); if (!(input instanceof PCollection) || ((PCollection) input).getWindowingStrategy().getWindowFn().isNonMerging()) { return false; @@ -338,7 +339,7 @@ TransformEvaluator translate( //--- determine if node is bounded/unbounded. // usually, the input determines if the PCollection to apply the next transformation to // is BOUNDED or UNBOUNDED, meaning RDD/DStream. - Collection pValues; + Collection pValues; if (node.getInputs().isEmpty()) { // in case of a PBegin, it's the output. pValues = node.getOutputs(); @@ -353,15 +354,15 @@ TransformEvaluator translate( : translator.translateUnbounded(transformClass); } - private PCollection.IsBounded isBoundedCollection(Collection pValues) { + private PCollection.IsBounded isBoundedCollection(Collection pValues) { // anything that is not a PCollection, is BOUNDED. // For PCollections: // BOUNDED behaves as the Identity Element, BOUNDED + BOUNDED = BOUNDED // while BOUNDED + UNBOUNDED = UNBOUNDED. PCollection.IsBounded isBounded = PCollection.IsBounded.BOUNDED; - for (PValue pValue: pValues) { - if (pValue instanceof PCollection) { - isBounded = isBounded.and(((PCollection) pValue).isBounded()); + for (TaggedPValue pValue: pValues) { + if (pValue.getValue() instanceof PCollection) { + isBounded = isBounded.and(((PCollection) pValue.getValue()).isBounded()); } else { isBounded = isBounded.and(PCollection.IsBounded.BOUNDED); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 33d5231ac24f..29e7fcb1a727 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -37,6 +37,7 @@ 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.TaggedPValue; /** * Captures information about a collection of transformations and their @@ -84,10 +85,12 @@ public Node pushNode(String name, PInput input, PTransform transform) { */ public void finishSpecifyingInput() { // Inputs must be completely specified before they are consumed by a transform. - for (PValue inputValue : current.getInputs()) { - inputValue.finishSpecifying(); - checkState(producers.get(inputValue) != null, "Producer unknown for input %s", inputValue); - inputValue.finishSpecifying(); + for (TaggedPValue inputValue : current.getInputs()) { + inputValue.getValue().finishSpecifying(); + checkState( + producers.get(inputValue.getValue()) != null, + "Producer unknown for input %s", + inputValue.getValue()); } } @@ -103,9 +106,9 @@ public void finishSpecifyingInput() { */ public void setOutput(POutput output) { output.finishSpecifyingOutput(); - for (PValue value : output.expand()) { - if (!producers.containsKey(value)) { - producers.put(value, current); + for (TaggedPValue value : output.expand()) { + if (!producers.containsKey(value.getValue())) { + producers.put(value.getValue(), current); } } current.setOutput(output); @@ -133,8 +136,8 @@ Node getProducer(PValue produced) { */ List getProducingTransforms(POutput output) { List producingTransforms = new ArrayList<>(); - for (PValue value : output.expand()) { - Node producer = getProducer(value); + for (TaggedPValue value : output.expand()) { + Node producer = getProducer(value.getValue()); if (producer != null) { producingTransforms.add(producer); } @@ -238,8 +241,8 @@ public boolean isCompositeNode() { private boolean returnsOthersOutput() { PTransform transform = getTransform(); if (output != null) { - for (PValue outputValue : output.expand()) { - if (!getProducer(outputValue).getTransform().equals(transform)) { + for (TaggedPValue outputValue : output.expand()) { + if (!getProducer(outputValue.getValue()).getTransform().equals(transform)) { return true; } } @@ -256,8 +259,8 @@ public String getFullName() { } /** Returns the transform input, in unexpanded form. */ - public Collection getInputs() { - return input == null ? Collections.emptyList() : input.expand(); + public List getInputs() { + return input == null ? Collections.emptyList() : input.expand(); } /** @@ -273,8 +276,8 @@ private void setOutput(POutput output) { // Validate that a primitive transform produces only primitive output, and a composite // transform does not produce primitive output. Set outputProducers = new HashSet<>(); - for (PValue outputValue : output.expand()) { - outputProducers.add(getProducer(outputValue)); + for (TaggedPValue outputValue : output.expand()) { + outputProducers.add(getProducer(outputValue.getValue())); } if (outputProducers.contains(this) && outputProducers.size() != 1) { Set otherProducerNames = new HashSet<>(); @@ -296,8 +299,8 @@ private void setOutput(POutput output) { } /** Returns the transform output, in unexpanded form. */ - public Collection getOutputs() { - return output == null ? Collections.emptyList() : output.expand(); + public List getOutputs() { + return output == null ? Collections.emptyList() : output.expand(); } /** @@ -320,9 +323,9 @@ private void visit(PipelineVisitor visitor, Set visitedValues) { if (!isRootNode()) { // Visit inputs. - for (PValue inputValue : input.expand()) { - if (visitedValues.add(inputValue)) { - visitor.visitValue(inputValue, getProducer(inputValue)); + for (TaggedPValue inputValue : input.expand()) { + if (visitedValues.add(inputValue.getValue())) { + visitor.visitValue(inputValue.getValue(), getProducer(inputValue.getValue())); } } } @@ -342,9 +345,9 @@ private void visit(PipelineVisitor visitor, Set visitedValues) { if (!isRootNode()) { // Visit outputs. - for (PValue pValue : output.expand()) { - if (visitedValues.add(pValue)) { - visitor.visitValue(pValue, this); + for (TaggedPValue pValue : output.expand()) { + if (visitedValues.add(pValue.getValue())) { + visitor.visitValue(pValue.getValue(), this); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java index 67b819fddc2c..13d4ee1b3291 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/KeyedPCollectionTuple.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.transforms.join; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; @@ -28,7 +27,7 @@ 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 org.apache.beam.sdk.values.TaggedPValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; @@ -121,10 +120,10 @@ public OutputT apply( * any tag-specific information. */ @Override - public Collection expand() { - List> retval = new ArrayList<>(); + public List expand() { + List retval = new ArrayList<>(); for (TaggedKeyedPCollection taggedPCollection : keyedCollections) { - retval.add(taggedPCollection.pCollection); + retval.add(TaggedPValue.of(taggedPCollection.tupleTag, taggedPCollection.pCollection)); } return retval; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java index f1dbb374a2c3..9aa46150e618 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PBegin.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.values; -import java.util.Collection; import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO.Read; import org.apache.beam.sdk.transforms.Create; @@ -64,7 +64,7 @@ public Pipeline getPipeline() { } @Override - public Collection expand() { + public List expand() { // A PBegin contains no PValues. return Collections.emptyList(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java index 4c9e220ea0ef..e4bb7c52ec56 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java @@ -18,11 +18,10 @@ package org.apache.beam.sdk.values; import com.google.common.collect.ImmutableList; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; +import com.google.common.collect.Iterables; import java.util.Iterator; import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten; @@ -115,9 +114,9 @@ public PCollectionList and(PCollection pc) { "PCollections come from different Pipelines"); } return new PCollectionList<>(pipeline, - new ImmutableList.Builder>() + ImmutableList.builder() .addAll(pcollections) - .add(pc) + .add(Iterables.getOnlyElement(pc.expand())) .build()); } @@ -130,15 +129,16 @@ public PCollectionList and(PCollection pc) { * part of the same {@link Pipeline}. */ public PCollectionList and(Iterable> pcs) { - List> copy = new ArrayList<>(pcollections); + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(pcollections); for (PCollection pc : pcs) { if (pc.getPipeline() != pipeline) { throw new IllegalArgumentException( "PCollections come from different Pipelines"); } - copy.add(pc); + builder.add(Iterables.getOnlyElement(pc.expand())); } - return new PCollectionList<>(pipeline, copy); + return new PCollectionList<>(pipeline, builder.build()); } /** @@ -155,7 +155,9 @@ public int size() { * {@code [0..size()-1]}. */ public PCollection get(int index) { - return pcollections.get(index); + @SuppressWarnings("unchecked") // Type-safe by construction + PCollection value = (PCollection) pcollections.get(index).getValue(); + return value; } /** @@ -163,7 +165,13 @@ public PCollection get(int index) { * {@link PCollectionList}. */ public List> getAll() { - return pcollections; + ImmutableList.Builder> res = ImmutableList.builder(); + for (TaggedPValue value : pcollections) { + @SuppressWarnings("unchecked") // Type-safe by construction + PCollection typedValue = (PCollection) value.getValue(); + res.add(typedValue); + } + return res.build(); } /** @@ -192,15 +200,16 @@ public OutputT apply( // Internal details below here. final Pipeline pipeline; - final List> pcollections; + // ImmutableMap has a defined iteration order. + final List pcollections; PCollectionList(Pipeline pipeline) { - this(pipeline, new ArrayList>()); + this(pipeline, ImmutableList.of()); } - PCollectionList(Pipeline pipeline, List> pcollections) { + PCollectionList(Pipeline pipeline, List values) { this.pipeline = pipeline; - this.pcollections = Collections.unmodifiableList(pcollections); + this.pcollections = ImmutableList.copyOf(values); } @Override @@ -209,14 +218,16 @@ public Pipeline getPipeline() { } @Override - public Collection expand() { + public List expand() { return pcollections; } @Override public void recordAsOutput(AppliedPTransform transform) { int i = 0; - for (PCollection pc : pcollections) { + for (TaggedPValue tpv : pcollections) { + @SuppressWarnings("unchecked") + PCollection pc = (PCollection) tpv.getValue(); pc.recordAsOutput(transform, "out" + i); i++; } @@ -224,15 +235,29 @@ public void recordAsOutput(AppliedPTransform transform) { @Override public void finishSpecifying() { - for (PCollection pc : pcollections) { - pc.finishSpecifying(); + for (TaggedPValue pc : pcollections) { + pc.getValue().finishSpecifying(); } } @Override public void finishSpecifyingOutput() { - for (PCollection pc : pcollections) { - pc.finishSpecifyingOutput(); + for (TaggedPValue pc : pcollections) { + pc.getValue().finishSpecifyingOutput(); } } + + @Override + public boolean equals(Object other) { + if (!(other instanceof PCollectionList)) { + return false; + } + PCollectionList that = (PCollectionList) other; + return this.pipeline.equals(that.pipeline) && this.pcollections.equals(that.pcollections); + } + + @Override + public int hashCode() { + return Objects.hash(this.pipeline, this.pcollections); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java index 727d882d745e..6afe59e0e07c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java @@ -17,11 +17,13 @@ */ package org.apache.beam.sdk.values; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import java.util.Collection; import java.util.Collections; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -178,7 +180,7 @@ public OutputT apply( ///////////////////////////////////////////////////////////////////////////// // Internal details below here. - Pipeline pipeline; + final Pipeline pipeline; final Map, PCollection> pcollectionMap; PCollectionTuple(Pipeline pipeline) { @@ -232,8 +234,12 @@ public Pipeline getPipeline() { } @Override - public Collection expand() { - return pcollectionMap.values(); + public List expand() { + ImmutableList.Builder values = ImmutableList.builder(); + for (Map.Entry, PCollection> entry : pcollectionMap.entrySet()) { + values.add(TaggedPValue.of(entry.getKey(), entry.getValue())); + } + return values.build(); } @Override @@ -261,4 +267,18 @@ public void finishSpecifyingOutput() { pc.finishSpecifyingOutput(); } } + + @Override + public boolean equals(Object other) { + if (!(other instanceof PCollectionTuple)) { + return false; + } + PCollectionTuple that = (PCollectionTuple) other; + return this.pipeline.equals(that.pipeline) && this.pcollectionMap.equals(that.pcollectionMap); + } + + @Override + public int hashCode() { + return Objects.hash(this.pipeline, this.pcollectionMap); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java index 9e8cae4a299f..b4a3025befdb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.values; -import java.util.Collection; import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; @@ -36,7 +36,7 @@ public static PDone in(Pipeline pipeline) { } @Override - public Collection expand() { + public List expand() { // A PDone contains no PValues. return Collections.emptyList(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java index f938aeb526b6..a27b939a40b0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PInput.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.values; -import java.util.Collection; +import java.util.List; import org.apache.beam.sdk.Pipeline; /** @@ -43,7 +43,7 @@ public interface PInput { * *

      Not intended to be invoked directly by user code. */ - Collection expand(); + List expand(); /** * After building, finalizes this {@code PInput} to make it ready for diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java index 27a280f76b31..e5d4504d01e7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.values; -import java.util.Collection; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -45,7 +45,7 @@ public interface POutput { * *

      Not intended to be invoked directly by user code. */ - Collection expand(); + List expand(); /** * Records that this {@code POutput} is an output of the given diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java index 0cee2ca7eea6..e6dbaf7d7bdd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.values; +import java.util.List; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -36,4 +37,13 @@ public interface PValue extends POutput, PInput { *

      For internal use only. */ AppliedPTransform getProducingTransformInternal(); + + /** + * {@inheritDoc}. + * + *

      A {@link PValue} always expands into itself. Calling {@link #expand()} on a PValue is almost + * never appropriate. + */ + @Deprecated + List expand(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java index 685e32fe15b7..3a10d5d5431d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.values; -import java.util.Collection; import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -90,6 +90,11 @@ protected PValueBase() { */ private String name; + /** + * A local {@link TupleTag} used in the expansion of this {@link PValueBase}. + */ + private TupleTag tag = new TupleTag<>(); + /** * Whether this {@link PValueBase} has been finalized, and its core * properties, e.g., name, can no longer be changed. @@ -128,8 +133,8 @@ public boolean isFinishedSpecifyingInternal() { } @Override - public Collection expand() { - return Collections.singletonList(this); + public final List expand() { + return Collections.singletonList(TaggedPValue.of(tag, this)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java new file mode 100644 index 000000000000..458d16ff70cf --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java @@ -0,0 +1,42 @@ +/* + * 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.values; + +import com.google.auto.value.AutoValue; + +/** + * A (TupleTag, PValue) pair used in the expansion of a {@link PInput} or {@link POutput}. + */ +@AutoValue +public abstract class TaggedPValue { + public static TaggedPValue of(TupleTag tag, PValue value) { + return new AutoValue_TaggedPValue(tag, value); + } + + /** + * Returns the local tag associated with the {@link PValue}. + */ + public abstract TupleTag getTag(); + + /** + * Returns the {@link PValue}. + */ + public abstract PValue getValue(); +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index 2327459cb9a1..d790d3910577 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -22,7 +22,10 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; +import com.google.common.base.Function; +import com.google.common.collect.Lists; import java.util.HashSet; +import java.util.List; import java.util.Set; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.io.CountingSource; @@ -38,6 +41,7 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Rule; @@ -181,14 +185,16 @@ public Long apply(Long input) { assertThat(hierarchy.getCurrent(), equalTo(primitiveNode)); hierarchy.setOutput(created); hierarchy.popNode(); - assertThat(primitiveNode.getOutputs(), Matchers.containsInAnyOrder(created)); - assertThat(primitiveNode.getInputs(), Matchers.emptyIterable()); + assertThat( + fromTaggedValues(primitiveNode.getOutputs()), Matchers.containsInAnyOrder(created)); + assertThat(primitiveNode.getInputs(), Matchers.emptyIterable()); assertThat(primitiveNode.getTransform(), Matchers.>equalTo(read)); assertThat(primitiveNode.getEnclosingNode(), equalTo(compositeNode)); hierarchy.setOutput(created); // The composite is listed as outputting a PValue created by the contained primitive - assertThat(compositeNode.getOutputs(), Matchers.containsInAnyOrder(created)); + assertThat( + fromTaggedValues(compositeNode.getOutputs()), Matchers.containsInAnyOrder(created)); // The producer of that PValue is still the primitive in which it is first output assertThat(hierarchy.getProducer(created), equalTo(primitiveNode)); hierarchy.popNode(); @@ -226,4 +232,15 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { assertThat(visitedValuesInVisitor, Matchers.containsInAnyOrder(created, mapped)); assertThat(visitedValuesInVisitor, equalTo(visitedValues)); } + + private static List fromTaggedValues(List taggedValues) { + return Lists.transform( + taggedValues, + new Function() { + @Override + public PValue apply(TaggedPValue input) { + return input.getValue(); + } + }); + } } 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 3a47fc7149c4..fa8874c72dff 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 @@ -29,6 +29,7 @@ import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import static org.junit.Assert.assertArrayEquals; @@ -50,6 +51,7 @@ import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; @@ -86,6 +88,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.MutableDateTime; @@ -863,6 +866,37 @@ public void testParDoWithSideOutputsName() { outputs.get(sideOutputTagUnwritten).getName()); } + @Test + public void testMultiOutputAppliedMultipleTimesDifferentOutputs() { + pipeline.enableAbandonedNodeEnforcement(false); + PCollection longs = pipeline.apply(CountingInput.unbounded()); + + TupleTag mainOut = new TupleTag<>(); + final TupleTag sideOutOne = new TupleTag<>(); + final TupleTag sideOutTwo = new TupleTag<>(); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext cxt) { + cxt.output(cxt.element()); + cxt.sideOutput(sideOutOne, Long.toString(cxt.element())); + cxt.sideOutput(sideOutTwo, Long.valueOf(cxt.element()).intValue()); + } + }; + + ParDo.BoundMulti parDo = + ParDo.of(fn).withOutputTags(mainOut, TupleTagList.of(sideOutOne).and(sideOutTwo)); + PCollectionTuple firstApplication = longs.apply("first", parDo); + PCollectionTuple secondApplication = longs.apply("second", parDo); + assertThat(firstApplication, not(equalTo(secondApplication))); + assertThat( + firstApplication.getAll().keySet(), + Matchers.>containsInAnyOrder(mainOut, sideOutOne, sideOutTwo)); + assertThat( + secondApplication.getAll().keySet(), + Matchers.>containsInAnyOrder(mainOut, sideOutOne, sideOutTwo)); + } + @Test @Category(RunnableOnService.class) public void testParDoInCustomTransform() { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java index f76bf7e23bf5..2482f325fa09 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionListTest.java @@ -18,10 +18,22 @@ package org.apache.beam.sdk.values; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.testing.EqualsTester; import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.CountingInput; +import org.apache.beam.sdk.io.CountingInput.BoundedCountingInput; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -44,4 +56,109 @@ public void testEmptyListFailure() { + "or must first call empty(Pipeline)")); } } + + @Test + public void testIterationOrder() { + Pipeline p = TestPipeline.create(); + PCollection createOne = p.apply("CreateOne", Create.of(1L, 2L, 3L)); + PCollection boundedCount = p.apply("CountBounded", CountingInput.upTo(23L)); + PCollection unboundedCount = p.apply("CountUnbounded", CountingInput.unbounded()); + PCollection createTwo = p.apply("CreateTwo", Create.of(-1L, -2L)); + PCollection maxRecordsCount = + p.apply("CountLimited", CountingInput.unbounded().withMaxNumRecords(22L)); + + ImmutableList> counts = + ImmutableList.of(boundedCount, maxRecordsCount, unboundedCount); + // Build a PCollectionList from a list. This should have the same order as the input list. + PCollectionList pcList = PCollectionList.of(counts); + // Contains is the order-dependent matcher + assertThat( + pcList.getAll(), + contains(boundedCount, maxRecordsCount, unboundedCount)); + + // A list that is expanded with builder methods has the added value at the end + PCollectionList withOneCreate = pcList.and(createTwo); + assertThat( + withOneCreate.getAll(), contains(boundedCount, maxRecordsCount, unboundedCount, createTwo)); + + // Lists that are built entirely from the builder return outputs in the order they were added + PCollectionList fromEmpty = + PCollectionList.empty(p) + .and(unboundedCount) + .and(createOne) + .and(ImmutableList.of(boundedCount, maxRecordsCount)); + assertThat( + fromEmpty.getAll(), contains(unboundedCount, createOne, boundedCount, maxRecordsCount)); + + List expansion = fromEmpty.expand(); + // TaggedPValues are stable between expansions + assertThat(expansion, equalTo(fromEmpty.expand())); + // TaggedPValues are equivalent between equivalent lists + assertThat( + expansion, + equalTo( + PCollectionList.of(unboundedCount) + .and(createOne) + .and(boundedCount) + .and(maxRecordsCount) + .expand())); + + List> expectedList = + ImmutableList.of(unboundedCount, createOne, boundedCount, maxRecordsCount); + for (int i = 0; i < expansion.size(); i++) { + assertThat( + "Index " + i + " should have equal PValue", + expansion.get(i).getValue(), + Matchers.equalTo(expectedList.get(i))); + } + } + + @Test + public void testEquals() { + Pipeline p = TestPipeline.create(); + PCollection first = p.apply("Meta", Create.of("foo", "bar")); + PCollection second = p.apply("Pythonic", Create.of("spam, ham")); + PCollection third = p.apply("Syntactic", Create.of("eggs", "baz")); + + EqualsTester tester = new EqualsTester(); + tester.addEqualityGroup(PCollectionList.empty(p), PCollectionList.empty(p)); + tester.addEqualityGroup(PCollectionList.of(first).and(second)); + // Constructors should all produce equivalent + tester.addEqualityGroup( + PCollectionList.of(first).and(second).and(third), + PCollectionList.of(first).and(second).and(third), + PCollectionList.empty(p).and(first).and(second).and(third), + PCollectionList.of(ImmutableList.of(first, second, third)), + PCollectionList.of(first).and(ImmutableList.of(second, third)), + PCollectionList.of(ImmutableList.of(first, second)).and(third)); + // Order is considered + tester.addEqualityGroup(PCollectionList.of(first).and(third).and(second)); + tester.addEqualityGroup(PCollectionList.empty(TestPipeline.create())); + + tester.testEquals(); + } + + @Test + public void testExpansionOrderWithDuplicates() { + TestPipeline p = TestPipeline.create(); + BoundedCountingInput count = CountingInput.upTo(10L); + PCollection firstCount = p.apply("CountFirst", count); + PCollection secondCount = p.apply("CountSecond", count); + + PCollectionList counts = + PCollectionList.of(firstCount).and(secondCount).and(firstCount).and(firstCount); + + ImmutableList> expectedOrder = + ImmutableList.of(firstCount, secondCount, firstCount, firstCount); + PCollectionList reconstructed = PCollectionList.empty(p); + assertThat(counts.expand(), hasSize(4)); + for (int i = 0; i < 4; i++) { + PValue value = counts.expand().get(i).getValue(); + assertThat( + "Index " + i + " should be equal", value, + Matchers.equalTo(expectedOrder.get(i))); + reconstructed = reconstructed.and((PCollection) value); + } + assertThat(reconstructed, equalTo(counts)); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java index b5351daac4e2..7d767cfe21da 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java @@ -17,21 +17,31 @@ */ package org.apache.beam.sdk.values; +import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableMap; +import com.google.common.testing.EqualsTester; import java.io.Serializable; import java.util.Arrays; import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.CountingInput; 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.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -93,4 +103,64 @@ public void processElement(ProcessContext c) { pipeline.run(); } + @Test + public void testEquals() { + TestPipeline p = TestPipeline.create(); + TupleTag longTag = new TupleTag<>(); + PCollection longs = p.apply(CountingInput.unbounded()); + TupleTag strTag = new TupleTag<>(); + PCollection strs = p.apply(Create.of("foo", "bar")); + + EqualsTester tester = new EqualsTester(); + // Empty tuples in the same pipeline are equal + tester.addEqualityGroup(PCollectionTuple.empty(p), PCollectionTuple.empty(p)); + + tester.addEqualityGroup(PCollectionTuple.of(longTag, longs).and(strTag, strs), + PCollectionTuple.of(longTag, longs).and(strTag, strs)); + + tester.addEqualityGroup(PCollectionTuple.of(longTag, longs)); + tester.addEqualityGroup(PCollectionTuple.of(strTag, strs)); + + TestPipeline otherPipeline = TestPipeline.create(); + // Empty tuples in different pipelines are not equal + tester.addEqualityGroup(PCollectionTuple.empty(otherPipeline)); + tester.testEquals(); + } + + @Test + public void testExpandHasMatchingTags() { + TupleTag intTag = new TupleTag<>(); + TupleTag strTag = new TupleTag<>(); + TupleTag longTag = new TupleTag<>(); + + Pipeline p = TestPipeline.create(); + PCollection longs = p.apply(CountingInput.upTo(100L)); + PCollection strs = p.apply(Create.of("foo", "bar", "baz")); + PCollection ints = longs.apply(MapElements.via(new SimpleFunction() { + @Override + public Integer apply(Long input) { + return input.intValue(); + } + })); + + Map, PCollection> pcsByTag = + ImmutableMap., PCollection>builder() + .put(strTag, strs) + .put(intTag, ints) + .put(longTag, longs) + .build(); + PCollectionTuple tuple = + PCollectionTuple.of(intTag, ints).and(longTag, longs).and(strTag, strs); + assertThat(tuple.getAll(), equalTo(pcsByTag)); + PCollectionTuple reconstructed = PCollectionTuple.empty(p); + for (TaggedPValue taggedValue : tuple.expand()) { + TupleTag tag = taggedValue.getTag(); + PValue value = taggedValue.getValue(); + assertThat("The tag should map back to the value", tuple.get(tag), equalTo(value)); + assertThat(value, Matchers.equalTo(pcsByTag.get(tag))); + reconstructed = reconstructed.and(tag, (PCollection) value); + } + + assertThat(reconstructed, equalTo(tuple)); + } } From 35a02740748182ee52729d8bfb621a3c342b8312 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 20:09:25 -0800 Subject: [PATCH 268/279] Require TimeDomain to delete a timer --- .../operators/ApexGroupByKeyOperator.java | 8 ++++++++ .../runners/core/InMemoryTimerInternals.java | 8 ++++++++ .../runners/direct/DirectTimerInternals.java | 8 ++++++++ .../wrappers/streaming/WindowDoFnOperator.java | 9 +++++++++ .../apache/beam/sdk/util/TimerInternals.java | 17 +++++++++++++++-- 5 files changed, 48 insertions(+), 2 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index 48ac177ead7b..49ec1c818c6e 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -425,11 +425,18 @@ protected Aggregator createAggreg */ public class ApexTimerInternals implements TimerInternals { + @Deprecated @Override public void setTimer(TimerData timerData) { registerActiveTimer(context.element().key(), timerData); } + @Override + public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) { + throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported."); + } + + @Deprecated @Override public void deleteTimer(TimerData timerKey) { unregisterActiveTimer(context.element().key(), timerKey); @@ -463,6 +470,7 @@ public void setTimer(StateNamespace namespace, String timerId, Instant target, throw new UnsupportedOperationException("Setting timer by ID not yet supported."); } + @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported."); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index 5fcd088c9400..5ddd5a7f813d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -107,6 +107,7 @@ public void setTimer(StateNamespace namespace, String timerId, Instant target, throw new UnsupportedOperationException("Setting a timer by ID is not yet supported."); } + @Deprecated @Override public void setTimer(TimerData timerData) { WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData); @@ -115,11 +116,18 @@ public void setTimer(TimerData timerData) { } } + @Override + public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) { + throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported."); + } + + @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported."); } + @Deprecated @Override public void deleteTimer(TimerData timer) { WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index 8970b4b88b8d..5ca276de56c0 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -52,16 +52,24 @@ public void setTimer(StateNamespace namespace, String timerId, Instant target, throw new UnsupportedOperationException("Setting timer by ID not yet supported."); } + @Deprecated @Override public void setTimer(TimerData timerData) { timerUpdateBuilder.setTimer(timerData); } + @Override + public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) { + throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported."); + } + + @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported."); } + @Deprecated @Override public void deleteTimer(TimerData timerKey) { timerUpdateBuilder.deletedTimer(timerKey); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 9cea5296d12b..5398d7b7d02a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -455,6 +455,7 @@ public void setTimer( throw new UnsupportedOperationException("Setting a timer by ID is not yet supported."); } + @Deprecated @Override public void setTimer(TimerData timerKey) { if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) { @@ -467,12 +468,20 @@ public void setTimer(TimerData timerKey) { } } + @Override + public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) { + throw new UnsupportedOperationException( + "Canceling of a timer by ID is not yet supported."); + } + + @Deprecated @Override public void deleteTimer(StateNamespace namespace, String timerId) { throw new UnsupportedOperationException( "Canceling of a timer by ID is not yet supported."); } + @Deprecated @Override public void deleteTimer(TimerData timerKey) { if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java index c3e498e03ae6..0bfcddc3197b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; @@ -61,18 +62,30 @@ public interface TimerInternals { void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain); /** - * Sets the timer described by {@code timerData}. + * @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */ + @Deprecated void setTimer(TimerData timerData); /** * Deletes the given timer. + * + *

      A timer's ID is enforced to be unique in validation of a {@link DoFn}, but runners + * often manage timers for different time domains in very different ways, thus the + * {@link TimeDomain} is a required parameter. + */ + void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain); + + /** + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */ + @Deprecated void deleteTimer(StateNamespace namespace, String timerId); /** - * Deletes the timer with the ID contained in the provided {@link TimerData}. + * @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */ + @Deprecated void deleteTimer(TimerData timerKey); /** From 64336e40dd6a48b3b6b48634bb9204db0aa0c7ca Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 21 Dec 2016 10:09:49 -0800 Subject: [PATCH 269/279] Update Dataflow worker to beam-master-20161221 --- .../org/apache/beam/runners/dataflow/dataflow.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index bf08e8394c3c..2912f618f064 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -18,6 +18,6 @@ environment.major.version=6 -worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161220 +worker.image.batch=dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20161221 -worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161220 +worker.image.streaming=dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20161221 From 9d2b8e09bcb5e04017b487e1a919d335875dbfc0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 15 Dec 2016 20:20:34 -0800 Subject: [PATCH 270/279] Move ExecutionContext and related classes to runners-core --- .../apex/translation/operators/ApexParDoOperator.java | 2 +- .../runners/apex/translation/utils/NoOpStepContext.java | 3 +-- .../org/apache/beam/runners/core/AggregatorFactory.java | 1 - .../org/apache/beam/runners/core}/BaseExecutionContext.java | 4 +++- .../main/java/org/apache/beam/runners/core/DoFnRunners.java | 2 +- .../org/apache/beam/runners/core}/ExecutionContext.java | 4 +++- .../java/org/apache/beam/runners/core/SimpleDoFnRunner.java | 2 +- .../org/apache/beam/runners/core/SimpleOldDoFnRunner.java | 2 +- .../org/apache/beam/runners/core/SimpleDoFnRunnerTest.java | 2 +- .../apache/beam/runners/core/SimpleOldDoFnRunnerTest.java | 3 +-- .../org/apache/beam/runners/direct/AggregatorContainer.java | 2 +- .../apache/beam/runners/direct/DirectExecutionContext.java | 6 +++--- .../org/apache/beam/runners/direct/EvaluationContext.java | 2 +- .../apache/beam/runners/direct/AggregatorContainerTest.java | 2 +- .../flink/translation/wrappers/streaming/DoFnOperator.java | 2 +- .../translation/wrappers/streaming/WindowDoFnOperator.java | 2 +- .../beam/runners/spark/aggregators/SparkAggregators.java | 2 +- .../beam/runners/spark/translation/SparkProcessContext.java | 2 +- 18 files changed, 23 insertions(+), 22 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/BaseExecutionContext.java (97%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/ExecutionContext.java (95%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java index a3d3a97e2ce0..c41cd454a66d 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java @@ -41,6 +41,7 @@ import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.sdk.coders.Coder; @@ -50,7 +51,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.UserCodeException; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java index 078f95fb1438..f169ae61e168 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java @@ -19,10 +19,9 @@ import java.io.IOException; import java.io.Serializable; - +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateInternals; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java index 153d30d67383..24a605f73774 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AggregatorFactory.java @@ -20,7 +20,6 @@ import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.ExecutionContext; /** * A factory for creating aggregators. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java index e26f2b0f62a5..7b674dc1d5da 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import java.io.IOException; import java.util.Collection; @@ -24,6 +24,8 @@ import java.util.Map; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index 0e4bf75ba0b4..820bfcd3076e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -19,13 +19,13 @@ import java.util.List; import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor; +import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java similarity index 95% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java index 4429d7614eab..f67aff4fc67d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java @@ -15,12 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import java.io.IOException; import java.util.Collection; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index a7d82bf52ed8..b42c57d45683 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -27,6 +27,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; @@ -47,7 +48,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimeDomain; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 342a4a869458..1ff02122119d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Set; import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimeDomain; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index ec5d375117e9..8ae09cb9bbac 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -26,13 +26,13 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.beam.runners.core.BaseExecutionContext.StepContext; import org.apache.beam.sdk.coders.Coder; 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.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java index 0e23dcbc649d..4610069aa64d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleOldDoFnRunnerTest.java @@ -22,9 +22,8 @@ import java.util.Arrays; import java.util.List; - +import org.apache.beam.runners.core.BaseExecutionContext.StepContext; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java index c7fa4df7cd8a..fd177044cd75 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AggregatorContainer.java @@ -28,9 +28,9 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import org.apache.beam.runners.core.AggregatorFactory; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.ExecutionContext; /** * AccumT container for the current values associated with {@link Aggregator Aggregators}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index c6051f08dfb1..8250cf11537b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -17,11 +17,11 @@ */ package org.apache.beam.runners.direct; +import org.apache.beam.runners.core.BaseExecutionContext; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; -import org.apache.beam.sdk.util.BaseExecutionContext; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.TimerInternals; /** @@ -54,7 +54,7 @@ protected DirectStepContext createStepContext(String stepName, String transformN * Step Context for the {@link DirectRunner}. */ public class DirectStepContext - extends org.apache.beam.sdk.util.BaseExecutionContext.StepContext { + extends BaseExecutionContext.StepContext { private CopyOnAccessInMemoryStateInternals stateInternals; private DirectTimerInternals timerInternals; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index bbcab8e4b8a0..3b9367a72229 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals.TimerData; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java index c8310c90e79a..f770800af674 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AggregatorContainerTest.java @@ -24,9 +24,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Sum.SumIntegerFn; -import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 870430844a7c..057a3e746696 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -31,6 +31,7 @@ import org.apache.beam.runners.core.AggregatorFactory; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; @@ -46,7 +47,6 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 9cea5296d12b..9855d4648029 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -38,6 +38,7 @@ import java.util.Set; import java.util.concurrent.ScheduledFuture; import javax.annotation.Nullable; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItems; @@ -48,7 +49,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java index 17d584481bc2..fa5c8d15c02a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/SparkAggregators.java @@ -22,11 +22,11 @@ import java.util.Collection; import java.util.Map; import org.apache.beam.runners.core.AggregatorFactory; +import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index 3a31caed94bc..9957bf330650 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -24,11 +24,11 @@ import java.util.Iterator; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.StateInternals; From df2e540d7a7b8444b9ff3b404740d5a3394b7691 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 19 Dec 2016 14:01:36 -0800 Subject: [PATCH 271/279] Support set and delete of timer by ID in InMemoryTimerInternals --- .../runners/core/InMemoryTimerInternals.java | 65 ++++++---- .../core/InMemoryTimerInternalsTest.java | 112 ++++++++++++------ 2 files changed, 120 insertions(+), 57 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index 5fcd088c9400..292ac23f65d0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -17,13 +17,15 @@ */ package org.apache.beam.runners.core; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import com.google.common.base.MoreObjects; -import java.util.HashSet; -import java.util.PriorityQueue; -import java.util.Set; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Table; +import java.util.NavigableSet; +import java.util.TreeSet; import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.TimeDomain; @@ -35,17 +37,17 @@ /** {@link TimerInternals} with all watermarks and processing clock simulated in-memory. */ public class InMemoryTimerInternals implements TimerInternals { - /** At most one timer per timestamp is kept. */ - private Set existingTimers = new HashSet<>(); + /** The current set timers by namespace and ID. */ + Table existingTimers = HashBasedTable.create(); /** Pending input watermark timers, in timestamp order. */ - private PriorityQueue watermarkTimers = new PriorityQueue<>(11); + private NavigableSet watermarkTimers = new TreeSet<>(); /** Pending processing time timers, in timestamp order. */ - private PriorityQueue processingTimers = new PriorityQueue<>(11); + private NavigableSet processingTimers = new TreeSet<>(); /** Pending synchronized processing time timers, in timestamp order. */ - private PriorityQueue synchronizedProcessingTimers = new PriorityQueue<>(11); + private NavigableSet synchronizedProcessingTimers = new TreeSet<>(); /** Current input watermark. */ private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; @@ -74,13 +76,13 @@ public Instant getNextTimer(TimeDomain domain) { final TimerData data; switch (domain) { case EVENT_TIME: - data = watermarkTimers.peek(); + data = watermarkTimers.first(); break; case PROCESSING_TIME: - data = processingTimers.peek(); + data = processingTimers.first(); break; case SYNCHRONIZED_PROCESSING_TIME: - data = synchronizedProcessingTimers.peek(); + data = synchronizedProcessingTimers.first(); break; default: throw new IllegalArgumentException("Unexpected time domain: " + domain); @@ -88,7 +90,7 @@ public Instant getNextTimer(TimeDomain domain) { return (data == null) ? null : data.getTimestamp(); } - private PriorityQueue queue(TimeDomain domain) { + private NavigableSet timersForDomain(TimeDomain domain) { switch (domain) { case EVENT_TIME: return watermarkTimers; @@ -104,27 +106,45 @@ private PriorityQueue queue(TimeDomain domain) { @Override public void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) { - throw new UnsupportedOperationException("Setting a timer by ID is not yet supported."); + setTimer(TimerData.of(timerId, namespace, target, timeDomain)); } @Override public void setTimer(TimerData timerData) { WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData); - if (existingTimers.add(timerData)) { - queue(timerData.getDomain()).add(timerData); + + @Nullable + TimerData existing = existingTimers.get(timerData.getNamespace(), timerData.getTimerId()); + if (existing == null) { + existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData); + timersForDomain(timerData.getDomain()).add(timerData); + } else { + checkArgument(timerData.getDomain().equals(existing.getDomain()), + "Attempt to set %s for time domain %s, but it is already set for time domain %s", + timerData.getTimerId(), timerData.getDomain(), existing.getDomain()); + + if (!timerData.getTimestamp().equals(existing.getTimestamp())) { + NavigableSet timers = timersForDomain(timerData.getDomain()); + timers.remove(existing); + timers.add(timerData); + existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData); + } } } @Override public void deleteTimer(StateNamespace namespace, String timerId) { - throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported."); + TimerData existing = existingTimers.get(namespace, timerId); + if (existing != null) { + deleteTimer(existing); + } } @Override public void deleteTimer(TimerData timer) { WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer); - existingTimers.remove(timer); - queue(timer.getDomain()).remove(timer); + existingTimers.remove(timer.getNamespace(), timer.getTimerId()); + timersForDomain(timer.getDomain()).remove(timer); } @Override @@ -261,10 +281,11 @@ public TimerData removeNextSynchronizedProcessingTimer() { @Nullable private TimerData removeNextTimer(Instant currentTime, TimeDomain domain) { - PriorityQueue queue = queue(domain); - if (!queue.isEmpty() && currentTime.isAfter(queue.peek().getTimestamp())) { - TimerData timer = queue.remove(); - existingTimers.remove(timer); + NavigableSet timers = timersForDomain(domain); + + if (!timers.isEmpty() && currentTime.isAfter(timers.first().getTimestamp())) { + TimerData timer = timers.pollFirst(); + existingTimers.remove(timer.getNamespace(), timer.getTimerId()); return timer; } else { return null; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java index 2caa8742dea1..e711285ae2fa 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java @@ -19,8 +19,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import org.apache.beam.sdk.util.TimeDomain; @@ -39,37 +37,79 @@ public class InMemoryTimerInternalsTest { private static final StateNamespace NS1 = new StateNamespaceForTest("NS1"); + private static final String ID1 = "id1"; + private static final String ID2 = "id2"; @Test - public void testFiringTimers() throws Exception { + public void testFiringEventTimers() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); - TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME); - TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME); + TimerData eventTimer1 = TimerData.of(ID1, NS1, new Instant(19), TimeDomain.EVENT_TIME); + TimerData eventTimer2 = TimerData.of(ID2, NS1, new Instant(29), TimeDomain.EVENT_TIME); - underTest.setTimer(processingTime1); - underTest.setTimer(processingTime2); + underTest.setTimer(eventTimer1); + underTest.setTimer(eventTimer2); - underTest.advanceProcessingTime(new Instant(20)); - assertEquals(processingTime1, underTest.removeNextProcessingTimer()); - assertNull(underTest.removeNextProcessingTimer()); + underTest.advanceInputWatermark(new Instant(20)); + assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer1)); + assertThat(underTest.removeNextEventTimer(), nullValue()); // Advancing just a little shouldn't refire - underTest.advanceProcessingTime(new Instant(21)); - assertNull(underTest.removeNextProcessingTimer()); + underTest.advanceInputWatermark(new Instant(21)); + assertThat(underTest.removeNextEventTimer(), nullValue()); // Adding the timer and advancing a little should refire - underTest.setTimer(processingTime1); - assertEquals(processingTime1, underTest.removeNextProcessingTimer()); - assertNull(underTest.removeNextProcessingTimer()); + underTest.setTimer(eventTimer1); + assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer1)); + assertThat(underTest.removeNextEventTimer(), nullValue()); // And advancing the rest of the way should still have the other timer - underTest.advanceProcessingTime(new Instant(30)); - assertEquals(processingTime2, underTest.removeNextProcessingTimer()); - assertNull(underTest.removeNextProcessingTimer()); + underTest.advanceInputWatermark(new Instant(30)); + assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer2)); + assertThat(underTest.removeNextEventTimer(), nullValue()); + } + + @Test + public void testResetById() throws Exception { + InMemoryTimerInternals underTest = new InMemoryTimerInternals(); + Instant earlyTimestamp = new Instant(13); + Instant laterTimestamp = new Instant(42); + + underTest.advanceInputWatermark(new Instant(0)); + underTest.setTimer(NS1, ID1, earlyTimestamp, TimeDomain.EVENT_TIME); + underTest.setTimer(NS1, ID1, laterTimestamp, TimeDomain.EVENT_TIME); + underTest.advanceInputWatermark(earlyTimestamp.plus(1L)); + assertThat(underTest.removeNextEventTimer(), nullValue()); + + underTest.advanceInputWatermark(laterTimestamp.plus(1L)); + assertThat( + underTest.removeNextEventTimer(), + equalTo(TimerData.of(ID1, NS1, laterTimestamp, TimeDomain.EVENT_TIME))); + } + + @Test + public void testDeletionIdempotent() throws Exception { + InMemoryTimerInternals underTest = new InMemoryTimerInternals(); + Instant timestamp = new Instant(42); + underTest.setTimer(NS1, ID1, timestamp, TimeDomain.EVENT_TIME); + underTest.deleteTimer(NS1, ID1); + underTest.deleteTimer(NS1, ID1); } @Test - public void testFiringTimersWithCallback() throws Exception { + public void testDeletionById() throws Exception { + InMemoryTimerInternals underTest = new InMemoryTimerInternals(); + Instant timestamp = new Instant(42); + + underTest.advanceInputWatermark(new Instant(0)); + underTest.setTimer(NS1, ID1, timestamp, TimeDomain.EVENT_TIME); + underTest.deleteTimer(NS1, ID1); + underTest.advanceInputWatermark(new Instant(43)); + + assertThat(underTest.removeNextEventTimer(), nullValue()); + } + + @Test + public void testFiringProcessingTimeTimers() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME); TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME); @@ -116,23 +156,25 @@ public void testTimerOrdering() throws Exception { underTest.setTimer(eventTime2); underTest.setTimer(synchronizedProcessingTime2); - assertNull(underTest.removeNextEventTimer()); + assertThat(underTest.removeNextEventTimer(), nullValue()); underTest.advanceInputWatermark(new Instant(30)); - assertEquals(eventTime1, underTest.removeNextEventTimer()); - assertEquals(eventTime2, underTest.removeNextEventTimer()); - assertNull(underTest.removeNextEventTimer()); + assertThat(underTest.removeNextEventTimer(), equalTo(eventTime1)); + assertThat(underTest.removeNextEventTimer(), equalTo(eventTime2)); + assertThat(underTest.removeNextEventTimer(), nullValue()); - assertNull(underTest.removeNextProcessingTimer()); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); underTest.advanceProcessingTime(new Instant(30)); - assertEquals(processingTime1, underTest.removeNextProcessingTimer()); - assertEquals(processingTime2, underTest.removeNextProcessingTimer()); - assertNull(underTest.removeNextProcessingTimer()); + assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1)); + assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); - assertNull(underTest.removeNextSynchronizedProcessingTimer()); + assertThat(underTest.removeNextSynchronizedProcessingTimer(), nullValue()); underTest.advanceSynchronizedProcessingTime(new Instant(30)); - assertEquals(synchronizedProcessingTime1, underTest.removeNextSynchronizedProcessingTimer()); - assertEquals(synchronizedProcessingTime2, underTest.removeNextSynchronizedProcessingTimer()); - assertNull(underTest.removeNextProcessingTimer()); + assertThat( + underTest.removeNextSynchronizedProcessingTimer(), equalTo(synchronizedProcessingTime1)); + assertThat( + underTest.removeNextSynchronizedProcessingTimer(), equalTo(synchronizedProcessingTime2)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); } @Test @@ -147,9 +189,9 @@ public void testDeduplicate() throws Exception { underTest.advanceProcessingTime(new Instant(20)); underTest.advanceInputWatermark(new Instant(20)); - assertEquals(processingTime, underTest.removeNextProcessingTimer()); - assertNull(underTest.removeNextProcessingTimer()); - assertEquals(eventTime, underTest.removeNextEventTimer()); - assertNull(underTest.removeNextEventTimer()); + assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime)); + assertThat(underTest.removeNextProcessingTimer(), nullValue()); + assertThat(underTest.removeNextEventTimer(), equalTo(eventTime)); + assertThat(underTest.removeNextEventTimer(), nullValue()); } } From 4fb16e8fb9bb087c0975f38c54665634868cfed7 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 13:58:29 -0800 Subject: [PATCH 272/279] Fix windowing in direct runner Stateful ParDo --- .../direct/ParDoMultiOverrideFactory.java | 34 +++++++++++++++++-- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 2cea9993ae44..b35df8714409 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -34,8 +34,13 @@ import org.apache.beam.sdk.transforms.ParDo.BoundMulti; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; 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.PCollectionTuple; @@ -92,9 +97,12 @@ public GbkThenStatefulParDo(ParDo.BoundMulti, OutputT> underlyingP @Override public PCollectionTuple expand(PCollection> input) { + WindowingStrategy inputWindowingStrategy = input.getWindowingStrategy(); + // A KvCoder is required since this goes through GBK. Further, WindowedValueCoder // is not registered by default, so we explicitly set the relevant coders. - checkState(input.getCoder() instanceof KvCoder, + checkState( + input.getCoder() instanceof KvCoder, "Input to a %s using state requires a %s, but the coder was %s", ParDo.class.getSimpleName(), KvCoder.class.getSimpleName(), @@ -102,14 +110,27 @@ public PCollectionTuple expand(PCollection> input) { KvCoder kvCoder = (KvCoder) input.getCoder(); Coder keyCoder = kvCoder.getKeyCoder(); Coder windowCoder = - input.getWindowingStrategy().getWindowFn().windowCoder(); + inputWindowingStrategy.getWindowFn().windowCoder(); - PCollectionTuple outputs = + PCollection>> adjustedInput = input // Stash the original timestamps, etc, for when it is fed to the user's DoFn .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn())) .setCoder(KvCoder.of(keyCoder, WindowedValue.getFullCoder(kvCoder, windowCoder))) + // We are going to GBK to gather keys and windows but otherwise do not want + // to alter the flow of data. This entails: + // - trigger as fast as possible + // - maintain the full timestamps of elements + // - ensure this GBK holds to the minimum of those timestamps (via OutputTimeFn) + // - discard past panes as it is "just a stream" of elements + .apply( + Window.>>>triggering( + Repeatedly.forever(AfterPane.elementCountAtLeast(1))) + .discardingFiredPanes() + .withAllowedLateness(inputWindowingStrategy.getAllowedLateness()) + .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())) + // A full GBK to group by key _and_ window .apply("Group by key", GroupByKey.>>create()) @@ -117,6 +138,13 @@ public PCollectionTuple expand(PCollection> input) { .apply("To KeyedWorkItem", ParDo.of(new ToKeyedWorkItem())) .setCoder(KeyedWorkItemCoder.of(keyCoder, kvCoder, windowCoder)) + // Because of the intervening GBK, we may have abused the windowing strategy + // of the input, which should be transferred to the output in a straightforward manner + // according to what ParDo already does. + .setWindowingStrategyInternal(inputWindowingStrategy); + + PCollectionTuple outputs = + adjustedInput // Explode the resulting iterable into elements that are exactly the ones from // the input .apply("Stateful ParDo", new StatefulParDo<>(underlyingParDo, input)); From 55176c385cc802be42b5467fbb2dcc9a1c7467ea Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 15:59:45 -0800 Subject: [PATCH 273/279] Actually propagate and commit state in direct runner --- .../beam/runners/direct/StatefulParDoEvaluatorFactory.java | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 5f9d8f49ed94..003df0feb2e5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -233,6 +233,7 @@ public TransformResult>> finishBundle() throws Ex StepTransformResult.>>withHold( delegateResult.getTransform(), delegateResult.getWatermarkHold()) .withTimerUpdate(delegateResult.getTimerUpdate()) + .withState(delegateResult.getState()) .withAggregatorChanges(delegateResult.getAggregatorChanges()) .withMetricUpdates(delegateResult.getLogicalMetricUpdates()) .addOutput(Lists.newArrayList(delegateResult.getOutputBundles())); From 280a6a8f729cb382616ad65f71860b61277cbd6f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 19 Dec 2016 20:40:11 -0800 Subject: [PATCH 274/279] Add informative Instant formatter to BoundedWindow --- .../beam/sdk/transforms/windowing/BoundedWindow.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java index 3654074174c1..6da249581339 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java @@ -39,6 +39,18 @@ public abstract class BoundedWindow { public static final Instant TIMESTAMP_MAX_VALUE = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE)); + public static String formatTimestamp(Instant timestamp) { + if (timestamp.equals(TIMESTAMP_MIN_VALUE)) { + return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)"; + } else if (timestamp.equals(TIMESTAMP_MAX_VALUE)) { + return timestamp.toString() + " (TIMESTAMP_MAX_VALUE)"; + } else if (timestamp.equals(GlobalWindow.INSTANCE.maxTimestamp())) { + return timestamp.toString() + " (end of global window)"; + } else { + return timestamp.toString(); + } + } + /** * Returns the inclusive upper bound of timestamps for values in this window. */ From fa4958a6140eb00ceee08b2468f7d88f17538794 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 19 Dec 2016 20:40:47 -0800 Subject: [PATCH 275/279] Use informative Instant formatter in WatermarkHold --- .../beam/runners/core/WatermarkHold.java | 4 +++- .../transforms/windowing/BoundedWindow.java | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index 7f1afcc0a79c..5e5f44d9138e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -207,7 +207,9 @@ private Instant shift(Instant timestamp, W window) { Instant shifted = windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window); checkState(!shifted.isBefore(timestamp), "OutputTimeFn moved element from %s to earlier time %s for window %s", - timestamp, shifted, window); + BoundedWindow.formatTimestamp(timestamp), + BoundedWindow.formatTimestamp(shifted), + window); checkState(timestamp.isAfter(window.maxTimestamp()) || !shifted.isAfter(window.maxTimestamp()), "OutputTimeFn moved element from %s to %s which is beyond end of " diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java index 6da249581339..74223b586715 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java @@ -34,11 +34,30 @@ public abstract class BoundedWindow { // The min and max timestamps that won't overflow when they are converted to // usec. + + /** + * The minimum value for any Beam timestamp. Often referred to as "-infinity". + * + *

      This value and {@link #TIMESTAMP_MAX_VALUE} are chosen so that their + * microseconds-since-epoch can be safely represented with a {@code long}. + */ public static final Instant TIMESTAMP_MIN_VALUE = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE)); + + /** + * The maximum value for any Beam timestamp. Often referred to as "+infinity". + * + *

      This value and {@link #TIMESTAMP_MIN_VALUE} are chosen so that their + * microseconds-since-epoch can be safely represented with a {@code long}. + */ public static final Instant TIMESTAMP_MAX_VALUE = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE)); + /** + * Formats a {@link Instant} timestamp with additional Beam-specific metadata, such as indicating + * whether the timestamp is the end of the global window or one of the distinguished values {@link + * #TIMESTAMP_MIN_VALUE} or {@link #TIMESTAMP_MIN_VALUE}. + */ public static String formatTimestamp(Instant timestamp) { if (timestamp.equals(TIMESTAMP_MIN_VALUE)) { return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)"; From 8188040d930b1fa49efd4ed7d5f821d05d6f28ef Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 13:57:55 -0800 Subject: [PATCH 276/279] Add static Window.withOutputTimeFn to match build method --- .../org/apache/beam/sdk/transforms/windowing/Window.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index 0c430d0ddc50..1241abe57c90 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -222,6 +222,15 @@ public static Bound withAllowedLateness(Duration allowedLateness) { return new Bound(null).withAllowedLateness(allowedLateness); } + /** + * (Experimental) Override the default {@link OutputTimeFn}, to control + * the output timestamp of values output from a {@link GroupByKey} operation. + */ + @Experimental(Kind.OUTPUT_TIME) + public static Bound withOutputTimeFn(OutputTimeFn outputTimeFn) { + return new Bound(null).withOutputTimeFn(outputTimeFn); + } + /** * A {@code PTransform} that windows the elements of a {@code PCollection}, * into finite windows according to a user-specified {@code WindowFn}. From 4d71924ccda9dae97c7cc9535a9780df9457cc3f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 14:20:07 -0800 Subject: [PATCH 277/279] Add UsesTestStream for use with JUnit @Category --- .../beam/sdk/testing/UsesTestStream.java | 24 +++++++++++++++++++ .../beam/sdk/testing/TestStreamTest.java | 12 +++++----- 2 files changed, 30 insertions(+), 6 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.java new file mode 100644 index 000000000000..8debb465a603 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStream.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; + +/** + * Category tag for tests that use {@link TestStream}, which is not a part of the Beam model + * but a special feature currently only implemented by the direct runner. + */ +public interface UsesTestStream {} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index 64aeca3cfe21..c12e9f332499 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -69,7 +69,7 @@ public class TestStreamTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); @Test - @Category(NeedsRunner.class) + @Category({NeedsRunner.class, UsesTestStream.class}) public void testLateDataAccumulating() { Instant instant = new Instant(0); TestStream source = TestStream.create(VarIntCoder.of()) @@ -136,7 +136,7 @@ public Void apply(Iterable input) { } @Test - @Category(NeedsRunner.class) + @Category({NeedsRunner.class, UsesTestStream.class}) public void testProcessingTimeTrigger() { TestStream source = TestStream.create(VarLongCoder.of()) .addElements(TimestampedValue.of(1L, new Instant(1000L)), @@ -159,7 +159,7 @@ public void testProcessingTimeTrigger() { } @Test - @Category(NeedsRunner.class) + @Category({NeedsRunner.class, UsesTestStream.class}) public void testDiscardingMode() { TestStream stream = TestStream.create(StringUtf8Coder.of()) @@ -208,7 +208,7 @@ public void testDiscardingMode() { } @Test - @Category(NeedsRunner.class) + @Category({NeedsRunner.class, UsesTestStream.class}) public void testFirstElementLate() { Instant lateElementTimestamp = new Instant(-1_000_000); TestStream stream = @@ -238,7 +238,7 @@ public void testFirstElementLate() { } @Test - @Category(NeedsRunner.class) + @Category({NeedsRunner.class, UsesTestStream.class}) public void testElementsAtAlmostPositiveInfinity() { Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp(); TestStream stream = TestStream.create(StringUtf8Coder.of()) @@ -261,7 +261,7 @@ public void testElementsAtAlmostPositiveInfinity() { } @Test - @Category(NeedsRunner.class) + @Category({NeedsRunner.class, UsesTestStream.class}) public void testMultipleStreams() { TestStream stream = TestStream.create(StringUtf8Coder.of()) .addElements("foo", "bar") From 7f14c463acd2ae5b86ac81a9528ac4aa7dff765f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 7 Dec 2016 20:18:44 -0800 Subject: [PATCH 278/279] Allow setting timer by ID in DirectTimerInternals --- .../runners/direct/DirectTimerInternals.java | 2 +- .../beam/runners/direct/WatermarkManager.java | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index 5ca276de56c0..80e072158274 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -49,7 +49,7 @@ private DirectTimerInternals( @Override public void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) { - throw new UnsupportedOperationException("Setting timer by ID not yet supported."); + timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, timeDomain)); } @Deprecated diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index 7bed75107f4a..f7bafd16ca55 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -23,11 +23,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.collect.ComparisonChain; +import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.common.collect.SortedMultiset; +import com.google.common.collect.Table; import com.google.common.collect.TreeMultiset; import java.io.Serializable; import java.util.ArrayList; @@ -56,6 +58,7 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TaggedPValue; import org.joda.time.Instant; @@ -210,6 +213,10 @@ private static class AppliedPTransformInputWatermark implements Watermark { private final SortedMultiset> pendingElements; private final Map, NavigableSet> objectTimers; + // Entries in this table represent the authoritative timestamp for which + // a per-key-and-StateNamespace timer is set. + private final Map, Table> existingTimers; + private AtomicReference currentWatermark; public AppliedPTransformInputWatermark(Collection inputWatermarks) { @@ -222,6 +229,7 @@ public AppliedPTransformInputWatermark(Collection inputWate this.pendingElements = TreeMultiset.create(pendingBundleComparator); this.objectTimers = new HashMap<>(); + this.existingTimers = new HashMap<>(); currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); } @@ -276,14 +284,31 @@ private synchronized void updateTimers(TimerUpdate update) { keyTimers = new TreeSet<>(); objectTimers.put(update.key, keyTimers); } + Table existingTimersForKey = + existingTimers.get(update.key); + if (existingTimersForKey == null) { + existingTimersForKey = HashBasedTable.create(); + existingTimers.put(update.key, existingTimersForKey); + } + for (TimerData timer : update.setTimers) { if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) { + @Nullable + TimerData existingTimer = + existingTimersForKey.get(timer.getNamespace(), timer.getTimerId()); + + if (existingTimer != null) { + keyTimers.remove(existingTimer); + } keyTimers.add(timer); + existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer); } } + for (TimerData timer : update.deletedTimers) { if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) { keyTimers.remove(timer); + existingTimersForKey.remove(timer.getNamespace(), timer.getTimerId()); } } // We don't keep references to timers that have been fired and delivered via #getFiredTimers() From dfe2e62d103595583e3ca4594cc03885fe1bba16 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 20 Dec 2016 13:37:40 -0800 Subject: [PATCH 279/279] Hold output watermark according to pending timers --- .../beam/runners/direct/WatermarkManager.java | 59 +++++++++++++++---- 1 file changed, 48 insertions(+), 11 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index f7bafd16ca55..248fafdb6493 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -211,12 +211,18 @@ public static WatermarkUpdate fromTimestamps(Instant oldTime, Instant currentTim private static class AppliedPTransformInputWatermark implements Watermark { private final Collection inputWatermarks; private final SortedMultiset> pendingElements; - private final Map, NavigableSet> objectTimers; + + // This tracks only the quantity of timers at each timestamp, for quickly getting the cross-key + // minimum + private final SortedMultiset pendingTimers; // Entries in this table represent the authoritative timestamp for which // a per-key-and-StateNamespace timer is set. private final Map, Table> existingTimers; + // This per-key sorted set allows quick retrieval of timers that should fire for a key + private final Map, NavigableSet> objectTimers; + private AtomicReference currentWatermark; public AppliedPTransformInputWatermark(Collection inputWatermarks) { @@ -224,10 +230,13 @@ public AppliedPTransformInputWatermark(Collection inputWate // The ordering must order elements by timestamp, and must not compare two distinct elements // as equal. This is built on the assumption that any element added as a pending element will // be consumed without modifications. + // + // The same logic is applied for pending timers Ordering> pendingBundleComparator = new BundleByElementTimestampComparator().compound(Ordering.arbitrary()); this.pendingElements = TreeMultiset.create(pendingBundleComparator); + this.pendingTimers = TreeMultiset.create(); this.objectTimers = new HashMap<>(); this.existingTimers = new HashMap<>(); currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); @@ -278,6 +287,14 @@ private synchronized void removePending(CommittedBundle completed) { pendingElements.remove(completed); } + private synchronized Instant getEarliestTimerTimestamp() { + if (pendingTimers.isEmpty()) { + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } else { + return pendingTimers.firstEntry().getElement(); + } + } + private synchronized void updateTimers(TimerUpdate update) { NavigableSet keyTimers = objectTimers.get(update.key); if (keyTimers == null) { @@ -291,27 +308,43 @@ private synchronized void updateTimers(TimerUpdate update) { existingTimers.put(update.key, existingTimersForKey); } - for (TimerData timer : update.setTimers) { + for (TimerData timer : update.getSetTimers()) { + if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) { + @Nullable + TimerData existingTimer = + existingTimersForKey.get(timer.getNamespace(), timer.getTimerId()); + + if (existingTimer == null) { + pendingTimers.add(timer.getTimestamp()); + keyTimers.add(timer); + } else if (!existingTimer.equals(timer)) { + keyTimers.remove(existingTimer); + keyTimers.add(timer); + } // else the timer is already set identically, so noop + + existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer); + } + } + + for (TimerData timer : update.getDeletedTimers()) { if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) { @Nullable TimerData existingTimer = existingTimersForKey.get(timer.getNamespace(), timer.getTimerId()); if (existingTimer != null) { + pendingTimers.remove(existingTimer.getTimestamp()); keyTimers.remove(existingTimer); + existingTimersForKey.remove(existingTimer.getNamespace(), existingTimer.getTimerId()); } - keyTimers.add(timer); - existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer); } } - for (TimerData timer : update.deletedTimers) { + for (TimerData timer : update.getCompletedTimers()) { if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) { - keyTimers.remove(timer); - existingTimersForKey.remove(timer.getNamespace(), timer.getTimerId()); + pendingTimers.remove(timer.getTimestamp()); } } - // We don't keep references to timers that have been fired and delivered via #getFiredTimers() } private synchronized Map, List> extractFiredEventTimeTimers() { @@ -336,11 +369,12 @@ public synchronized String toString() { * {@link #refresh()} for more information. */ private static class AppliedPTransformOutputWatermark implements Watermark { - private final Watermark inputWatermark; + private final AppliedPTransformInputWatermark inputWatermark; private final PerKeyHolds holds; private AtomicReference currentWatermark; - public AppliedPTransformOutputWatermark(AppliedPTransformInputWatermark inputWatermark) { + public AppliedPTransformOutputWatermark( + AppliedPTransformInputWatermark inputWatermark) { this.inputWatermark = inputWatermark; holds = new PerKeyHolds(); currentWatermark = new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); @@ -377,7 +411,10 @@ public Instant get() { @Override public synchronized WatermarkUpdate refresh() { Instant oldWatermark = currentWatermark.get(); - Instant newWatermark = INSTANT_ORDERING.min(inputWatermark.get(), holds.getMinHold()); + Instant newWatermark = INSTANT_ORDERING.min( + inputWatermark.get(), + inputWatermark.getEarliestTimerTimestamp(), + holds.getMinHold()); newWatermark = INSTANT_ORDERING.max(oldWatermark, newWatermark); currentWatermark.set(newWatermark); return WatermarkUpdate.fromTimestamps(oldWatermark, newWatermark);